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

This commit is contained in:
proller 2019-07-26 12:31:34 +03:00
commit a49074f22b
114 changed files with 2560 additions and 868 deletions

View File

@ -7,6 +7,8 @@ set(SRCS
${RDKAFKA_SOURCE_DIR}/rdavl.c ${RDKAFKA_SOURCE_DIR}/rdavl.c
${RDKAFKA_SOURCE_DIR}/rdbuf.c ${RDKAFKA_SOURCE_DIR}/rdbuf.c
${RDKAFKA_SOURCE_DIR}/rdcrc32.c ${RDKAFKA_SOURCE_DIR}/rdcrc32.c
${RDKAFKA_SOURCE_DIR}/rddl.c
${RDKAFKA_SOURCE_DIR}/rdhdrhistogram.c
${RDKAFKA_SOURCE_DIR}/rdkafka.c ${RDKAFKA_SOURCE_DIR}/rdkafka.c
${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_background.c ${RDKAFKA_SOURCE_DIR}/rdkafka_background.c
@ -28,11 +30,13 @@ set(SRCS
${RDKAFKA_SOURCE_DIR}/rdkafka_op.c ${RDKAFKA_SOURCE_DIR}/rdkafka_op.c
${RDKAFKA_SOURCE_DIR}/rdkafka_partition.c ${RDKAFKA_SOURCE_DIR}/rdkafka_partition.c
${RDKAFKA_SOURCE_DIR}/rdkafka_pattern.c ${RDKAFKA_SOURCE_DIR}/rdkafka_pattern.c
${RDKAFKA_SOURCE_DIR}/rdkafka_plugin.c
${RDKAFKA_SOURCE_DIR}/rdkafka_queue.c ${RDKAFKA_SOURCE_DIR}/rdkafka_queue.c
${RDKAFKA_SOURCE_DIR}/rdkafka_range_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_range_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_request.c ${RDKAFKA_SOURCE_DIR}/rdkafka_request.c
${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_oauthbearer.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_plain.c
${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_scram.c ${RDKAFKA_SOURCE_DIR}/rdkafka_sasl_scram.c
${RDKAFKA_SOURCE_DIR}/rdkafka_ssl.c ${RDKAFKA_SOURCE_DIR}/rdkafka_ssl.c

View File

@ -16,7 +16,7 @@
#define MKL_APP_NAME "librdkafka" #define MKL_APP_NAME "librdkafka"
#define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library" #define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library"
// distro // distro
//#define SOLIB_EXT ".so" #define SOLIB_EXT ".so"
// gcc // gcc
//#define WITH_GCC 1 //#define WITH_GCC 1
// gxx // gxx
@ -48,9 +48,9 @@
// parseversion // parseversion
#define MKL_APP_VERSION "0.11.4" #define MKL_APP_VERSION "0.11.4"
// libdl // libdl
//#define WITH_LIBDL 1 #define WITH_LIBDL 1
// WITH_PLUGINS // WITH_PLUGINS
//#define WITH_PLUGINS 1 #define WITH_PLUGINS 1
// zlib // zlib
#define WITH_ZLIB 1 #define WITH_ZLIB 1
// zstd // zstd
@ -63,6 +63,8 @@
#define WITH_SSL 1 #define WITH_SSL 1
// WITH_SASL_SCRAM // WITH_SASL_SCRAM
#define WITH_SASL_SCRAM 1 #define WITH_SASL_SCRAM 1
// WITH_SASL_OAUTHBEARER
#define WITH_SASL_OAUTHBEARER 1
// crc32chw // crc32chw
#if !defined(__PPC__) #if !defined(__PPC__)
#define WITH_CRC32C_HW 1 #define WITH_CRC32C_HW 1
@ -78,5 +80,5 @@
// python // python
//#define HAVE_PYTHON 1 //#define HAVE_PYTHON 1
// disable C11 threads for compatibility with old libc // disable C11 threads for compatibility with old libc
#define WITH_C11THREADS 0 //#define WITH_C11THREADS 1
#endif /* _CONFIG_H_ */ #endif /* _CONFIG_H_ */

2
contrib/zlib-ng vendored

@ -1 +1 @@
Subproject commit 9173b89d46799582d20a30578e0aa9788bc7d6e1 Subproject commit cb43e7fa08ec29fd76d84e3bb35258a0f0bf3df3

View File

@ -20,6 +20,9 @@
#include <Common/StringUtils/StringUtils.h> #include <Common/StringUtils/StringUtils.h>
#include <common/phdr_cache.h>
/// Universal executable for various clickhouse applications /// Universal executable for various clickhouse applications
#if ENABLE_CLICKHOUSE_SERVER || !defined(ENABLE_CLICKHOUSE_SERVER) #if ENABLE_CLICKHOUSE_SERVER || !defined(ENABLE_CLICKHOUSE_SERVER)
int mainEntryClickHouseServer(int argc, char ** argv); int mainEntryClickHouseServer(int argc, char ** argv);
@ -144,6 +147,11 @@ int main(int argc_, char ** argv_)
/// It is needed because LLVM library clobbers it. /// It is needed because LLVM library clobbers it.
std::set_new_handler(nullptr); std::set_new_handler(nullptr);
/// PHDR cache is required for query profiler to work reliably
/// It also speed up exception handling, but exceptions from dynamically loaded libraries (dlopen)
/// will work only after additional call of this function.
updatePHDRCache();
#if USE_EMBEDDED_COMPILER #if USE_EMBEDDED_COMPILER
if (argc_ >= 2 && 0 == strcmp(argv_[1], "-cc1")) if (argc_ >= 2 && 0 == strcmp(argv_[1], "-cc1"))
return mainEntryClickHouseClang(argc_, argv_); return mainEntryClickHouseClang(argc_, argv_);

View File

@ -14,6 +14,7 @@
#include <Poco/Util/HelpFormatter.h> #include <Poco/Util/HelpFormatter.h>
#include <ext/scope_guard.h> #include <ext/scope_guard.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <common/phdr_cache.h>
#include <common/ErrorHandlers.h> #include <common/ErrorHandlers.h>
#include <common/getMemoryAmount.h> #include <common/getMemoryAmount.h>
#include <Common/ClickHouseRevision.h> #include <Common/ClickHouseRevision.h>
@ -509,7 +510,8 @@ int Server::main(const std::vector<std::string> & /*args*/)
LOG_DEBUG(log, "Loaded metadata."); LOG_DEBUG(log, "Loaded metadata.");
/// Init trace collector only after trace_log system table was created /// Init trace collector only after trace_log system table was created
global_context->initializeTraceCollector(); if (hasPHDRCache())
global_context->initializeTraceCollector();
global_context->setCurrentDatabase(default_database); global_context->setCurrentDatabase(default_database);

View File

@ -210,7 +210,13 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root,
void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with) void ConfigProcessor::merge(XMLDocumentPtr config, XMLDocumentPtr with)
{ {
mergeRecursive(config, getRootNode(&*config), getRootNode(&*with)); Node * config_root = getRootNode(config.get());
Node * with_root = getRootNode(with.get());
if (config_root->nodeName() != with_root->nodeName())
throw Poco::Exception("Root element doesn't have the corresponding root element as the config file. It must be <" + config_root->nodeName() + ">");
mergeRecursive(config, config_root, with_root);
} }
std::string ConfigProcessor::layerFromHost() std::string ConfigProcessor::layerFromHost()

View File

@ -437,6 +437,7 @@ namespace ErrorCodes
extern const int CANNOT_CREATE_TIMER = 460; extern const int CANNOT_CREATE_TIMER = 460;
extern const int CANNOT_SET_TIMER_PERIOD = 461; extern const int CANNOT_SET_TIMER_PERIOD = 461;
extern const int CANNOT_DELETE_TIMER = 462; extern const int CANNOT_DELETE_TIMER = 462;
extern const int CANNOT_FCNTL = 463;
extern const int KEEPER_EXCEPTION = 999; extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000; extern const int POCO_EXCEPTION = 1000;

View File

@ -171,6 +171,9 @@
M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \
M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \
M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \ M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \
\
M(QueryProfilerCannotWriteTrace, "Number of stack traces dropped by query profiler because pipe is full or cannot write to pipe.") \
M(QueryProfilerSignalOverruns, "Number of times we drop processing of a signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \
namespace ProfileEvents namespace ProfileEvents
{ {

View File

@ -1,14 +1,26 @@
#include "QueryProfiler.h" #include "QueryProfiler.h"
#include <random>
#include <pcg_random.hpp>
#include <common/Pipe.h> #include <common/Pipe.h>
#include <common/phdr_cache.h>
#include <common/config_common.h>
#include <common/StackTrace.h> #include <common/StackTrace.h>
#include <common/StringRef.h> #include <common/StringRef.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <Common/CurrentThread.h> #include <Common/CurrentThread.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromFileDescriptor.h> #include <IO/WriteBufferFromFileDescriptor.h>
namespace ProfileEvents
{
extern const Event QueryProfilerCannotWriteTrace;
extern const Event QueryProfilerSignalOverruns;
}
namespace DB namespace DB
{ {
@ -16,23 +28,76 @@ extern LazyPipe trace_pipe;
namespace namespace
{ {
/** Write to file descriptor but drop the data if write would block or fail.
* To use within signal handler. Motivating example: a signal handler invoked during execution of malloc
* should not block because some mutex (or even worse - a spinlock) may be held.
*/
class WriteBufferDiscardOnFailure : public WriteBufferFromFileDescriptor
{
protected:
void nextImpl() override
{
size_t bytes_written = 0;
while (bytes_written != offset())
{
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
if ((-1 == res || 0 == res) && errno != EINTR)
{
ProfileEvents::increment(ProfileEvents::QueryProfilerCannotWriteTrace);
break; /// Discard
}
if (res > 0)
bytes_written += res;
}
}
public:
using WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor;
~WriteBufferDiscardOnFailure() override {}
};
/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id. /// 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. /// 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; constexpr size_t QUERY_ID_MAX_LEN = 1024;
void writeTraceInfo(TimerType timer_type, int /* sig */, siginfo_t * /* info */, void * context) thread_local size_t write_trace_iteration = 0;
thread_local pcg64 rng{randomSeed()};
void writeTraceInfo(TimerType timer_type, int /* sig */, siginfo_t * info, void * context)
{ {
/// 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;
}
}
constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag constexpr size_t buf_size = sizeof(char) + // TraceCollector stop flag
8 * sizeof(char) + // maximum VarUInt length for string size 8 * sizeof(char) + // maximum VarUInt length for string size
QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length
sizeof(StackTrace) + // collected stack trace sizeof(StackTrace) + // collected stack trace
sizeof(TimerType); // timer type sizeof(TimerType) + // timer type
sizeof(UInt32); // thread_number
char buffer[buf_size]; char buffer[buf_size];
WriteBufferFromFileDescriptor out(trace_pipe.fds_rw[1], buf_size, buffer); WriteBufferDiscardOnFailure out(trace_pipe.fds_rw[1], buf_size, buffer);
StringRef query_id = CurrentThread::getQueryId(); StringRef query_id = CurrentThread::getQueryId();
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN); query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
UInt32 thread_number = CurrentThread::get().thread_number;
const auto signal_context = *reinterpret_cast<ucontext_t *>(context); const auto signal_context = *reinterpret_cast<ucontext_t *>(context);
const StackTrace stack_trace(signal_context); const StackTrace stack_trace(signal_context);
@ -40,6 +105,7 @@ namespace
writeStringBinary(query_id, out); writeStringBinary(query_id, out);
writePODBinary(stack_trace, out); writePODBinary(stack_trace, out);
writePODBinary(timer_type, out); writePODBinary(timer_type, out);
writePODBinary(thread_number, out);
out.next(); out.next();
} }
@ -56,10 +122,19 @@ namespace ErrorCodes
} }
template <typename ProfilerImpl> template <typename ProfilerImpl>
QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const Int32 thread_id, const int clock_type, const UInt32 period, const int pause_signal) QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const Int32 thread_id, const int clock_type, UInt32 period, const int pause_signal)
: log(&Logger::get("QueryProfiler")) : log(&Logger::get("QueryProfiler"))
, pause_signal(pause_signal) , pause_signal(pause_signal)
{ {
#if USE_INTERNAL_UNWIND_LIBRARY
/// Sanity check.
if (!hasPHDRCache())
throw Exception("QueryProfiler cannot be used without PHDR cache, that is not available for TSan build", ErrorCodes::NOT_IMPLEMENTED);
/// Too high frequency can introduce infinite busy loop of signal handlers. We will limit maximum frequency (with 1000 signals per second).
if (period < 1000000)
period = 1000000;
struct sigaction sa{}; struct sigaction sa{};
sa.sa_sigaction = ProfilerImpl::signalHandler; sa.sa_sigaction = ProfilerImpl::signalHandler;
sa.sa_flags = SA_SIGINFO | SA_RESTART; sa.sa_flags = SA_SIGINFO | SA_RESTART;
@ -87,8 +162,16 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const Int32 thread_id, const
if (timer_create(clock_type, &sev, &timer_id)) if (timer_create(clock_type, &sev, &timer_id))
throwFromErrno("Failed to create thread timer", ErrorCodes::CANNOT_CREATE_TIMER); throwFromErrno("Failed to create thread timer", ErrorCodes::CANNOT_CREATE_TIMER);
/// Randomize offset as uniform random value from 0 to period - 1.
/// It will allow to sample short queries even if timer period is large.
/// (For example, with period of 1 second, query with 50 ms duration will be sampled with 1 / 20 probability).
/// It also helps to avoid interference (moire).
UInt32 period_rand = std::uniform_int_distribution<UInt32>(0, period)(rng);
struct timespec interval{.tv_sec = period / TIMER_PRECISION, .tv_nsec = period % TIMER_PRECISION}; struct timespec interval{.tv_sec = period / TIMER_PRECISION, .tv_nsec = period % TIMER_PRECISION};
struct itimerspec timer_spec = {.it_interval = interval, .it_value = interval}; struct timespec offset{.tv_sec = period_rand / TIMER_PRECISION, .tv_nsec = period_rand % TIMER_PRECISION};
struct itimerspec timer_spec = {.it_interval = interval, .it_value = offset};
if (timer_settime(timer_id, 0, &timer_spec, nullptr)) if (timer_settime(timer_id, 0, &timer_spec, nullptr))
throwFromErrno("Failed to set thread timer period", ErrorCodes::CANNOT_SET_TIMER_PERIOD); throwFromErrno("Failed to set thread timer period", ErrorCodes::CANNOT_SET_TIMER_PERIOD);
} }
@ -97,6 +180,9 @@ QueryProfilerBase<ProfilerImpl>::QueryProfilerBase(const Int32 thread_id, const
tryCleanup(); tryCleanup();
throw; throw;
} }
#else
throw Exception("QueryProfiler cannot work with stock libunwind", ErrorCodes::NOT_IMPLEMENTED);
#endif
} }
template <typename ProfilerImpl> template <typename ProfilerImpl>

View File

@ -5,6 +5,7 @@
#include <signal.h> #include <signal.h>
#include <time.h> #include <time.h>
namespace Poco namespace Poco
{ {
class Logger; class Logger;
@ -34,8 +35,7 @@ template <typename ProfilerImpl>
class QueryProfilerBase class QueryProfilerBase
{ {
public: public:
QueryProfilerBase(const Int32 thread_id, const int clock_type, const UInt32 period, const int pause_signal = SIGALRM); QueryProfilerBase(const Int32 thread_id, const int clock_type, UInt32 period, const int pause_signal);
~QueryProfilerBase(); ~QueryProfilerBase();
private: private:

View File

@ -1,6 +1,7 @@
#include "SharedLibrary.h" #include "SharedLibrary.h"
#include <string> #include <string>
#include <boost/core/noncopyable.hpp> #include <boost/core/noncopyable.hpp>
#include <common/phdr_cache.h>
#include "Exception.h" #include "Exception.h"
@ -17,6 +18,8 @@ SharedLibrary::SharedLibrary(const std::string & path, int flags)
handle = dlopen(path.c_str(), flags); handle = dlopen(path.c_str(), flags);
if (!handle) if (!handle)
throw Exception(std::string("Cannot dlopen: ") + dlerror(), ErrorCodes::CANNOT_DLOPEN); throw Exception(std::string("Cannot dlopen: ") + dlerror(), ErrorCodes::CANNOT_DLOPEN);
updatePHDRCache();
} }
SharedLibrary::~SharedLibrary() SharedLibrary::~SharedLibrary()

View File

@ -12,6 +12,10 @@
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Interpreters/TraceLog.h> #include <Interpreters/TraceLog.h>
#include <unistd.h>
#include <fcntl.h>
namespace DB namespace DB
{ {
@ -21,6 +25,7 @@ namespace ErrorCodes
{ {
extern const int NULL_POINTER_DEREFERENCE; extern const int NULL_POINTER_DEREFERENCE;
extern const int THREAD_IS_NOT_JOINABLE; extern const int THREAD_IS_NOT_JOINABLE;
extern const int CANNOT_FCNTL;
} }
TraceCollector::TraceCollector(std::shared_ptr<TraceLog> & trace_log) TraceCollector::TraceCollector(std::shared_ptr<TraceLog> & trace_log)
@ -31,6 +36,28 @@ TraceCollector::TraceCollector(std::shared_ptr<TraceLog> & trace_log)
throw Exception("Invalid trace log pointer passed", ErrorCodes::NULL_POINTER_DEREFERENCE); throw Exception("Invalid trace log pointer passed", ErrorCodes::NULL_POINTER_DEREFERENCE);
trace_pipe.open(); trace_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.
*/
int flags = fcntl(trace_pipe.fds_rw[1], F_GETFL, 0);
if (-1 == flags)
throwFromErrno("Cannot get file status flags of pipe", ErrorCodes::CANNOT_FCNTL);
if (-1 == fcntl(trace_pipe.fds_rw[1], F_SETFL, flags | O_NONBLOCK))
throwFromErrno("Cannot set non-blocking mode of pipe", ErrorCodes::CANNOT_FCNTL);
/** Increase pipe size to avoid slowdown during fine-grained trace collection.
*/
constexpr int max_pipe_capacity_to_set = 1048576;
int pipe_size = fcntl(trace_pipe.fds_rw[1], F_GETPIPE_SZ);
if (-1 == pipe_size)
throwFromErrno("Cannot get pipe capacity", ErrorCodes::CANNOT_FCNTL);
for (errno = 0; errno != EPERM && pipe_size < max_pipe_capacity_to_set; pipe_size *= 2)
if (-1 == fcntl(trace_pipe.fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM)
throwFromErrno("Cannot increase pipe capacity to " + toString(pipe_size * 2), ErrorCodes::CANNOT_FCNTL);
LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, max_pipe_capacity_to_set)));
thread = ThreadFromGlobalPool(&TraceCollector::run, this); thread = ThreadFromGlobalPool(&TraceCollector::run, this);
} }
@ -78,10 +105,12 @@ void TraceCollector::run()
std::string query_id; std::string query_id;
StackTrace stack_trace(NoCapture{}); StackTrace stack_trace(NoCapture{});
TimerType timer_type; TimerType timer_type;
UInt32 thread_number;
readStringBinary(query_id, in); readStringBinary(query_id, in);
readPODBinary(stack_trace, in); readPODBinary(stack_trace, in);
readPODBinary(timer_type, in); readPODBinary(timer_type, in);
readPODBinary(thread_number, in);
const auto size = stack_trace.getSize(); const auto size = stack_trace.getSize();
const auto & frames = stack_trace.getFrames(); const auto & frames = stack_trace.getFrames();
@ -91,7 +120,7 @@ void TraceCollector::run()
for (size_t i = 0; i < size; i++) for (size_t i = 0; i < size; i++)
trace.emplace_back(UInt64(reinterpret_cast<uintptr_t>(frames[i]))); trace.emplace_back(UInt64(reinterpret_cast<uintptr_t>(frames[i])));
TraceLogElement element{std::time(nullptr), timer_type, query_id, trace}; TraceLogElement element{std::time(nullptr), timer_type, thread_number, query_id, trace};
trace_log->add(element); trace_log->add(element);
} }

View File

@ -3,8 +3,7 @@
#include <cstdint> #include <cstdint>
#include <algorithm> #include <algorithm>
#ifdef __SSE2__ #include <Core/Defines.h>
#include <emmintrin.h>
namespace detail namespace detail
@ -22,6 +21,15 @@ inline int cmp(T a, T b)
} }
/// We can process uninitialized memory in the functions below.
/// Results don't depend on the values inside uninitialized memory but Memory Sanitizer cannot see it.
/// Disable optimized functions if compile with Memory Sanitizer.
#if defined(__SSE2__) && !defined(MEMORY_SANITIZER)
#include <emmintrin.h>
/** All functions works under the following assumptions: /** All functions works under the following assumptions:
* - it's possible to read up to 15 excessive bytes after end of 'a' and 'b' region; * - it's possible to read up to 15 excessive bytes after end of 'a' and 'b' region;
* - memory regions are relatively small and extra loop unrolling is not worth to do. * - memory regions are relatively small and extra loop unrolling is not worth to do.
@ -192,7 +200,10 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size)
template <typename Char> template <typename Char>
inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size) inline int memcmpSmallAllowOverflow15(const Char * a, size_t a_size, const Char * b, size_t b_size)
{ {
return memcmp(a, b, std::min(a_size, b_size)); if (auto res = memcmp(a, b, std::min(a_size, b_size)))
return res;
else
return detail::cmp(a_size, b_size);
} }
template <typename Char> template <typename Char>

View File

@ -7,7 +7,7 @@
/// Replace default new/delete with memory tracking versions. /// Replace default new/delete with memory tracking versions.
/// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new
/// https://en.cppreference.com/w/cpp/memory/new/operator_delete /// https://en.cppreference.com/w/cpp/memory/new/operator_delete
#if NOT_UNBUNDLED #if !UNBUNDLED
namespace Memory namespace Memory
{ {

View File

@ -34,6 +34,7 @@ const size_t SSL_REQUEST_PAYLOAD_SIZE = 32;
namespace Authentication namespace Authentication
{ {
const String Native = "mysql_native_password";
const String SHA256 = "sha256_password"; /// Caching SHA2 plugin is not used because it would be possible to authenticate knowing hash from users.xml. const String SHA256 = "sha256_password"; /// Caching SHA2 plugin is not used because it would be possible to authenticate knowing hash from users.xml.
} }
@ -285,7 +286,12 @@ public:
result.append(1, auth_plugin_data.size()); result.append(1, auth_plugin_data.size());
result.append(10, 0x0); result.append(10, 0x0);
result.append(auth_plugin_data.substr(AUTH_PLUGIN_DATA_PART_1_LENGTH, auth_plugin_data.size() - AUTH_PLUGIN_DATA_PART_1_LENGTH)); result.append(auth_plugin_data.substr(AUTH_PLUGIN_DATA_PART_1_LENGTH, auth_plugin_data.size() - AUTH_PLUGIN_DATA_PART_1_LENGTH));
result.append(Authentication::SHA256);
// A workaround for PHP mysqlnd extension bug which occurs when sha256_password is used as a default authentication plugin.
// Instead of using client response for mysql_native_password plugin, the server will always generate authentication method mismatch
// and switch to sha256_password to simulate that mysql_native_password is used as a default plugin.
result.append(Authentication::Native);
result.append(1, 0x0); result.append(1, 0x0);
return result; return result;
} }

View File

@ -221,8 +221,8 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.") \ M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.") \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.") \ M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.") \
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.") \ M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.") \
M(SettingUInt64, query_profiler_real_time_period_ns, 0, "Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off real clock query profiler") \ M(SettingUInt64, query_profiler_real_time_period_ns, 0, "Highly experimental. Period for real clock timer of query profiler (in nanoseconds). Set 0 value to turn off real clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.") \
M(SettingUInt64, query_profiler_cpu_time_period_ns, 0, "Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off CPU clock query profiler") \ M(SettingUInt64, query_profiler_cpu_time_period_ns, 0, "Highly experimental. Period for CPU clock timer of query profiler (in nanoseconds). Set 0 value to turn off CPU clock query profiler. Recommended value is at least 10000000 (100 times a second) for single queries or 1000000000 (once a second) for cluster-wide profiling.") \
\ \
\ \
/** Limits during query execution are part of the settings. \ /** Limits during query execution are part of the settings. \
@ -307,6 +307,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \ M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \
M(SettingLogsLevel, send_logs_level, LogsLevel::none, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \ M(SettingLogsLevel, send_logs_level, LogsLevel::none, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \
M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.") \ M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.") \
M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.") \
\ \
M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \ M(SettingUInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.") \
M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \ M(SettingBool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.") \
@ -338,7 +339,7 @@ struct Settings : public SettingsCollection<Settings>
\ \
/** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \ /** Obsolete settings that do nothing but left for compatibility reasons. Remove each one after half a year of obsolescence. */ \
\ \
M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") M(SettingBool, allow_experimental_low_cardinality_type, true, "Obsolete setting, does nothing. Will be removed after 2019-08-13") \
DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS) DECLARE_SETTINGS_COLLECTION(LIST_OF_SETTINGS)

View File

@ -462,7 +462,7 @@ void DataTypeArray::deserializeProtobuf(IColumn & column, ProtobufReader & proto
bool nested_row_added; bool nested_row_added;
do do
nested->deserializeProtobuf(nested_column, protobuf, true, nested_row_added); nested->deserializeProtobuf(nested_column, protobuf, true, nested_row_added);
while (nested_row_added && protobuf.maybeCanReadValue()); while (nested_row_added && protobuf.canReadMoreValues());
if (allow_add_row) if (allow_add_row)
{ {
offsets.emplace_back(nested_column.size()); offsets.emplace_back(nested_column.size());

View File

@ -34,19 +34,18 @@ namespace
BITS32 = 5, BITS32 = 5,
}; };
// The following should be always true: // The following condition must always be true:
// REACHED_END < any cursor position < min(END_OF_VARINT, END_OF_GROUP) // any_cursor_position < min(END_OF_VARINT, END_OF_GROUP)
// This inequation helps to check conditions in SimpleReader. // This inequation helps to check conditions in SimpleReader.
constexpr UInt64 END_OF_VARINT = static_cast<UInt64>(-1); constexpr UInt64 END_OF_VARINT = static_cast<UInt64>(-1);
constexpr UInt64 END_OF_GROUP = static_cast<UInt64>(-2); constexpr UInt64 END_OF_GROUP = static_cast<UInt64>(-2);
Int64 decodeZigZag(UInt64 n) { return static_cast<Int64>((n >> 1) ^ (~(n & 1) + 1)); } Int64 decodeZigZag(UInt64 n) { return static_cast<Int64>((n >> 1) ^ (~(n & 1) + 1)); }
}
[[noreturn]] void throwUnknownFormat()
[[noreturn]] void ProtobufReader::SimpleReader::throwUnknownFormat() {
{ throw Exception("Protobuf messages are corrupted or don't match the provided schema. Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint.", ErrorCodes::UNKNOWN_PROTOBUF_FORMAT);
throw Exception("Protobuf messages are corrupted or don't match the provided schema. Please note that Protobuf stream is length-delimited: every message is prefixed by its length in varint.", ErrorCodes::UNKNOWN_PROTOBUF_FORMAT); }
} }
@ -54,94 +53,103 @@ namespace
// Knows nothing about protobuf schemas, just provides useful functions to deserialize data. // Knows nothing about protobuf schemas, just provides useful functions to deserialize data.
ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_) ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_)
: in(in_) : in(in_)
, cursor(1 /* We starts at cursor == 1 to keep any cursor value > REACHED_END, this allows to simplify conditions */) , cursor(0)
, current_message_end(REACHED_END) , current_message_level(0)
, field_end(REACHED_END) , current_message_end(0)
, field_end(0)
, last_string_pos(-1)
{ {
} }
bool ProtobufReader::SimpleReader::startMessage() bool ProtobufReader::SimpleReader::startMessage()
{ {
if ((current_message_end == REACHED_END) && parent_message_ends.empty()) // Start reading a root message.
{ assert(!current_message_level);
// Start reading a root message. if (unlikely(in.eof()))
if (unlikely(in.eof())) return false;
return false; size_t size_of_message = readVarint();
size_t size_of_message = readVarint(); current_message_end = cursor + size_of_message;
if (size_of_message == 0) ++current_message_level;
throwUnknownFormat(); field_end = cursor;
current_message_end = cursor + size_of_message;
root_message_end = current_message_end;
}
else
{
// Start reading a nested message which is located inside a length-delimited field
// of another message.s
parent_message_ends.emplace_back(current_message_end);
current_message_end = field_end;
}
field_end = REACHED_END;
return true; return true;
} }
void ProtobufReader::SimpleReader::endMessage() void ProtobufReader::SimpleReader::endMessage(bool ignore_errors)
{ {
if (current_message_end != REACHED_END) if (!current_message_level)
return;
UInt64 root_message_end = (current_message_level == 1) ? current_message_end : parent_message_ends.front();
if (cursor != root_message_end)
{ {
if (current_message_end == END_OF_GROUP) if (cursor < root_message_end)
ignoreGroup(); ignore(root_message_end - cursor);
else if (cursor < current_message_end) else if (ignore_errors)
ignore(current_message_end - cursor); moveCursorBackward(cursor - root_message_end);
else if (unlikely(cursor > current_message_end)) else
{ throwUnknownFormat();
if (!parent_message_ends.empty())
throwUnknownFormat();
moveCursorBackward(cursor - current_message_end);
}
current_message_end = REACHED_END;
} }
field_end = REACHED_END; current_message_level = 0;
if (!parent_message_ends.empty()) parent_message_ends.clear();
{
current_message_end = parent_message_ends.back();
parent_message_ends.pop_back();
}
} }
void ProtobufReader::SimpleReader::endRootMessage() void ProtobufReader::SimpleReader::startNestedMessage()
{ {
UInt64 message_end = parent_message_ends.empty() ? current_message_end : parent_message_ends.front(); assert(current_message_level >= 1);
if (message_end != REACHED_END) // Start reading a nested message which is located inside a length-delimited field
// of another message.
parent_message_ends.emplace_back(current_message_end);
current_message_end = field_end;
++current_message_level;
field_end = cursor;
}
void ProtobufReader::SimpleReader::endNestedMessage()
{
assert(current_message_level >= 2);
if (cursor != current_message_end)
{ {
if (cursor < message_end) if (current_message_end == END_OF_GROUP)
ignore(message_end - cursor); {
else if (unlikely(cursor > message_end)) ignoreGroup();
moveCursorBackward(cursor - message_end); current_message_end = cursor;
}
else if (cursor < current_message_end)
ignore(current_message_end - cursor);
else
throwUnknownFormat();
} }
parent_message_ends.clear();
current_message_end = REACHED_END; --current_message_level;
field_end = REACHED_END; current_message_end = parent_message_ends.back();
parent_message_ends.pop_back();
field_end = cursor;
} }
bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number) bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number)
{ {
if (field_end != REACHED_END) assert(current_message_level);
if (field_end != cursor)
{ {
if (field_end == END_OF_VARINT) if (field_end == END_OF_VARINT)
{
ignoreVarint(); ignoreVarint();
field_end = cursor;
}
else if (field_end == END_OF_GROUP) else if (field_end == END_OF_GROUP)
{
ignoreGroup(); ignoreGroup();
field_end = cursor;
}
else if (cursor < field_end) else if (cursor < field_end)
ignore(field_end - cursor); ignore(field_end - cursor);
field_end = REACHED_END; else
throwUnknownFormat();
} }
if (cursor >= current_message_end) if (cursor >= current_message_end)
{
current_message_end = REACHED_END;
return false; return false;
}
UInt64 varint = readVarint(); UInt64 varint = readVarint();
if (unlikely(varint & (static_cast<UInt64>(0xFFFFFFFF) << 32))) if (unlikely(varint & (static_cast<UInt64>(0xFFFFFFFF) << 32)))
@ -151,6 +159,11 @@ bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number)
WireType wire_type = static_cast<WireType>(key & 0x07); WireType wire_type = static_cast<WireType>(key & 0x07);
switch (wire_type) switch (wire_type)
{ {
case BITS32:
{
field_end = cursor + 4;
return true;
}
case BITS64: case BITS64:
{ {
field_end = cursor + 8; field_end = cursor + 8;
@ -176,29 +189,20 @@ bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number)
{ {
if (current_message_end != END_OF_GROUP) if (current_message_end != END_OF_GROUP)
throwUnknownFormat(); throwUnknownFormat();
current_message_end = REACHED_END; current_message_end = cursor;
return false; return false;
} }
case BITS32:
{
field_end = cursor + 4;
return true;
}
} }
throwUnknownFormat(); throwUnknownFormat();
__builtin_unreachable();
} }
bool ProtobufReader::SimpleReader::readUInt(UInt64 & value) bool ProtobufReader::SimpleReader::readUInt(UInt64 & value)
{ {
if (unlikely(cursor >= field_end)) if (unlikely(cursor >= field_end))
{
field_end = REACHED_END;
return false; return false;
}
value = readVarint(); value = readVarint();
if ((field_end == END_OF_VARINT) || (cursor >= field_end)) if (field_end == END_OF_VARINT)
field_end = REACHED_END; field_end = cursor;
return true; return true;
} }
@ -224,25 +228,22 @@ template<typename T>
bool ProtobufReader::SimpleReader::readFixed(T & value) bool ProtobufReader::SimpleReader::readFixed(T & value)
{ {
if (unlikely(cursor >= field_end)) if (unlikely(cursor >= field_end))
{
field_end = REACHED_END;
return false; return false;
}
readBinary(&value, sizeof(T)); readBinary(&value, sizeof(T));
if (cursor >= field_end)
field_end = REACHED_END;
return true; return true;
} }
bool ProtobufReader::SimpleReader::readStringInto(PaddedPODArray<UInt8> & str) bool ProtobufReader::SimpleReader::readStringInto(PaddedPODArray<UInt8> & str)
{ {
if (unlikely(cursor == last_string_pos))
return false; /// We don't want to read the same empty string again.
last_string_pos = cursor;
if (unlikely(cursor > field_end)) if (unlikely(cursor > field_end))
return false; throwUnknownFormat();
size_t length = field_end - cursor; size_t length = field_end - cursor;
size_t old_size = str.size(); size_t old_size = str.size();
str.resize(old_size + length); str.resize(old_size + length);
readBinary(reinterpret_cast<char*>(str.data() + old_size), length); readBinary(reinterpret_cast<char*>(str.data() + old_size), length);
field_end = REACHED_END;
return true; return true;
} }
@ -299,7 +300,6 @@ UInt64 ProtobufReader::SimpleReader::continueReadingVarint(UInt64 first_byte)
#undef PROTOBUF_READER_READ_VARINT_BYTE #undef PROTOBUF_READER_READ_VARINT_BYTE
throwUnknownFormat(); throwUnknownFormat();
__builtin_unreachable();
} }
void ProtobufReader::SimpleReader::ignoreVarint() void ProtobufReader::SimpleReader::ignoreVarint()
@ -1083,9 +1083,9 @@ bool ProtobufReader::startMessage()
return true; return true;
} }
void ProtobufReader::endMessage() void ProtobufReader::endMessage(bool try_ignore_errors)
{ {
simple_reader.endRootMessage(); simple_reader.endMessage(try_ignore_errors);
current_message = nullptr; current_message = nullptr;
current_converter = nullptr; current_converter = nullptr;
} }
@ -1102,7 +1102,7 @@ bool ProtobufReader::readColumnIndex(size_t & column_index)
current_converter = nullptr; current_converter = nullptr;
return false; return false;
} }
simple_reader.endMessage(); simple_reader.endNestedMessage();
current_field_index = current_message->index_in_parent; current_field_index = current_message->index_in_parent;
current_message = current_message->parent; current_message = current_message->parent;
continue; continue;
@ -1132,7 +1132,7 @@ bool ProtobufReader::readColumnIndex(size_t & column_index)
if (field->nested_message) if (field->nested_message)
{ {
simple_reader.startMessage(); simple_reader.startNestedMessage();
current_message = field->nested_message.get(); current_message = field->nested_message.get();
current_field_index = 0; current_field_index = 0;
continue; continue;

View File

@ -42,7 +42,7 @@ public:
bool startMessage(); bool startMessage();
/// Ends reading a message. /// Ends reading a message.
void endMessage(); void endMessage(bool ignore_errors = false);
/// Reads the column index. /// Reads the column index.
/// The function returns false if there are no more columns to read (call endMessage() in this case). /// The function returns false if there are no more columns to read (call endMessage() in this case).
@ -79,9 +79,8 @@ public:
bool readAggregateFunction(const AggregateFunctionPtr & function, AggregateDataPtr place, Arena & arena) { return current_converter->readAggregateFunction(function, place, arena); } bool readAggregateFunction(const AggregateFunctionPtr & function, AggregateDataPtr place, Arena & arena) { return current_converter->readAggregateFunction(function, place, arena); }
/// When it returns false there is no more values left and from now on all the read<Type>() functions will return false /// Call it after calling one of the read*() function to determine if there are more values available for reading.
/// until readColumnIndex() is called. When it returns true it's unclear. bool ALWAYS_INLINE canReadMoreValues() const { return simple_reader.canReadMoreValues(); }
bool ALWAYS_INLINE maybeCanReadValue() const { return simple_reader.maybeCanReadValue(); }
private: private:
class SimpleReader class SimpleReader
@ -89,8 +88,9 @@ private:
public: public:
SimpleReader(ReadBuffer & in_); SimpleReader(ReadBuffer & in_);
bool startMessage(); bool startMessage();
void endMessage(); void endMessage(bool ignore_errors);
void endRootMessage(); void startNestedMessage();
void endNestedMessage();
bool readFieldNumber(UInt32 & field_number); bool readFieldNumber(UInt32 & field_number);
bool readInt(Int64 & value); bool readInt(Int64 & value);
bool readSInt(Int64 & value); bool readSInt(Int64 & value);
@ -98,15 +98,7 @@ private:
template<typename T> bool readFixed(T & value); template<typename T> bool readFixed(T & value);
bool readStringInto(PaddedPODArray<UInt8> & str); bool readStringInto(PaddedPODArray<UInt8> & str);
bool ALWAYS_INLINE maybeCanReadValue() const bool ALWAYS_INLINE canReadMoreValues() const { return cursor < field_end; }
{
if (field_end == REACHED_END)
return false;
if (cursor < root_message_end)
return true;
throwUnknownFormat();
}
private: private:
void readBinary(void * data, size_t size); void readBinary(void * data, size_t size);
@ -128,17 +120,13 @@ private:
void ignoreVarint(); void ignoreVarint();
void ignoreGroup(); void ignoreGroup();
[[noreturn]] static void throwUnknownFormat();
static constexpr UInt64 REACHED_END = 0;
ReadBuffer & in; ReadBuffer & in;
UInt64 cursor; UInt64 cursor;
std::vector<UInt64> parent_message_ends; size_t current_message_level;
UInt64 current_message_end; UInt64 current_message_end;
std::vector<UInt64> parent_message_ends;
UInt64 field_end; UInt64 field_end;
UInt64 last_string_pos;
UInt64 root_message_end;
}; };
class IConverter class IConverter

View File

@ -41,7 +41,7 @@ bool ProtobufRowInputStream::read(MutableColumns & columns, RowReadExtension & e
read_columns[column_index] = true; read_columns[column_index] = true;
allow_add_row = false; allow_add_row = false;
} }
} while (reader.maybeCanReadValue()); } while (reader.canReadMoreValues());
} }
// Fill non-visited columns with the default values. // Fill non-visited columns with the default values.
@ -60,7 +60,7 @@ bool ProtobufRowInputStream::allowSyncAfterError() const
void ProtobufRowInputStream::syncAfterError() void ProtobufRowInputStream::syncAfterError()
{ {
reader.endMessage(); reader.endMessage(true);
} }

View File

@ -141,7 +141,8 @@ void ProtobufWriter::SimpleWriter::endMessage()
size_t size_of_message = buffer.size() - num_bytes_skipped; size_t size_of_message = buffer.size() - num_bytes_skipped;
writeVarint(size_of_message, out); writeVarint(size_of_message, out);
for (const auto & piece : pieces) for (const auto & piece : pieces)
out.write(reinterpret_cast<char *>(&buffer[piece.start]), piece.end - piece.start); if (piece.end > piece.start)
out.write(reinterpret_cast<char *>(&buffer[piece.start]), piece.end - piece.start);
buffer.clear(); buffer.clear();
pieces.clear(); pieces.clear();
num_bytes_skipped = 0; num_bytes_skipped = 0;

View File

@ -1,12 +0,0 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsIntrospection.h>
namespace DB
{
void registerFunctionsIntrospection(FunctionFactory & factory)
{
factory.registerFunction<FunctionSymbolizeTrace>();
}
}

View File

@ -1,107 +0,0 @@
#pragma once
#include <common/StackTrace.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
class FunctionSymbolizeTrace : public IFunction
{
public:
static constexpr auto name = "symbolizeTrace";
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionSymbolizeTrace>();
}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 1;
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " needs exactly one argument; passed "
+ toString(arguments.size()) + ".",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto array_type = checkAndGetDataType<DataTypeArray>(arguments[0].type.get());
if (!array_type)
throw Exception("The only argument for function " + getName() + " must be array. Found "
+ arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
DataTypePtr nested_type = array_type->getNestedType();
if (!WhichDataType(nested_type).isUInt64())
throw Exception("The only argument for function " + getName() + " must be array of UInt64. Found "
+ arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override
{
return true;
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
const ColumnPtr column = block.getByPosition(arguments[0]).column;
const ColumnArray * column_array = checkAndGetColumn<ColumnArray>(column.get());
if (!column_array)
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
const ColumnPtr data_ptr = column_array->getDataPtr();
const ColumnVector<UInt64> * data_vector = checkAndGetColumn<ColumnVector<UInt64>>(&*data_ptr);
const typename ColumnVector<UInt64>::Container & data = data_vector->getData();
const ColumnArray::Offsets & offsets = column_array->getOffsets();
auto result_column = ColumnString::create();
StackTrace::Frames frames;
size_t current_offset = 0;
for (size_t i = 0; i < offsets.size(); ++i)
{
size_t current_size = 0;
for (; current_size < frames.size() && current_offset + current_size < offsets[i]; ++current_size)
{
frames[current_size] = reinterpret_cast<void *>(data[current_offset + current_size]);
}
std::string backtrace = StackTrace(frames.begin(), frames.begin() + current_size).toString();
result_column->insertDataWithTerminatingZero(backtrace.c_str(), backtrace.length() + 1);
current_offset = offsets[i];
}
block.getByPosition(result).column = std::move(result_column);
}
};
}

View File

@ -40,6 +40,7 @@ void registerFunctionsIntrospection(FunctionFactory &);
void registerFunctionsNull(FunctionFactory &); void registerFunctionsNull(FunctionFactory &);
void registerFunctionsFindCluster(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &);
void registerFunctionsJSON(FunctionFactory &); void registerFunctionsJSON(FunctionFactory &);
void registerFunctionSymbolizeAddress(FunctionFactory &);
void registerFunctions() void registerFunctions()
{ {
@ -75,10 +76,10 @@ void registerFunctions()
registerFunctionsVisitParam(factory); registerFunctionsVisitParam(factory);
registerFunctionsMath(factory); registerFunctionsMath(factory);
registerFunctionsGeo(factory); registerFunctionsGeo(factory);
registerFunctionsIntrospection(factory);
registerFunctionsNull(factory); registerFunctionsNull(factory);
registerFunctionsFindCluster(factory); registerFunctionsFindCluster(factory);
registerFunctionsJSON(factory); registerFunctionsJSON(factory);
registerFunctionSymbolizeAddress(factory);
} }
} }

View File

@ -0,0 +1,115 @@
#include <dlfcn.h>
#include <unordered_map>
#include <optional>
#include <common/unaligned.h>
#include <common/demangle.h>
#include <common/SimpleCache.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/FunctionFactory.h>
#include <IO/WriteHelpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DOESNT_MATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
class FunctionSymbolizeAddress : public IFunction
{
public:
static constexpr auto name = "symbolizeAddress";
static FunctionPtr create(const Context &)
{
return std::make_shared<FunctionSymbolizeAddress>();
}
String getName() const override
{
return name;
}
size_t getNumberOfArguments() const override
{
return 1;
}
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.size() != 1)
throw Exception("Function " + getName() + " needs exactly one argument; passed "
+ toString(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const auto & type = arguments[0].type;
if (!WhichDataType(type.get()).isUInt64())
throw Exception("The only argument for function " + getName() + " must be UInt64. Found "
+ type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();
}
bool useDefaultImplementationForConstants() const override
{
return true;
}
static std::string addressToSymbol(UInt64 uint_address)
{
void * addr = unalignedLoad<void *>(&uint_address);
/// This is extremely slow.
Dl_info info;
if (dladdr(addr, &info) && info.dli_sname)
{
int demangling_status = 0;
return demangle(info.dli_sname, demangling_status);
}
else
{
return {};
}
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
{
const ColumnPtr & column = block.getByPosition(arguments[0]).column;
const ColumnUInt64 * column_concrete = checkAndGetColumn<ColumnUInt64>(column.get());
if (!column_concrete)
throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN);
const typename ColumnVector<UInt64>::Container & data = column_concrete->getData();
auto result_column = ColumnString::create();
static SimpleCache<decltype(addressToSymbol), &addressToSymbol> func_cached;
for (size_t i = 0; i < input_rows_count; ++i)
{
std::string symbol = func_cached(data[i]);
result_column->insertDataWithTerminatingZero(symbol.data(), symbol.size() + 1);
}
block.getByPosition(result).column = std::move(result_column);
/// Do not let our cache to grow indefinitely (simply drop it)
if (func_cached.size() > 1000000)
func_cached.drop();
}
};
void registerFunctionSymbolizeAddress(FunctionFactory & factory)
{
factory.registerFunction<FunctionSymbolizeAddress>();
}
}

View File

@ -21,6 +21,11 @@ public:
return typeid_cast<BufferType *>(buffer.get()); return typeid_cast<BufferType *>(buffer.get());
} }
void reset()
{
BufferBase::set(nullptr, 0, 0);
}
protected: protected:
// XXX: don't know how to guarantee that the next call to this method is done after we read all previous data. // XXX: don't know how to guarantee that the next call to this method is done after we read all previous data.
bool nextImpl() override bool nextImpl() override

View File

@ -0,0 +1,196 @@
#pragma once
#include <Parsers/ASTFunction.h>
#include <Parsers/queryToString.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/SyntaxAnalyzer.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_JOIN_ON_EXPRESSION;
extern const int AMBIGUOUS_COLUMN_NAME;
extern const int LOGICAL_ERROR;
}
class CollectJoinOnKeysMatcher
{
public:
using Visitor = ConstInDepthNodeVisitor<CollectJoinOnKeysMatcher, true>;
struct Data
{
AnalyzedJoin & analyzed_join;
const NameSet & source_columns;
const NameSet & joined_columns;
const Aliases & aliases;
bool has_some = false;
};
static void visit(const ASTPtr & ast, Data & data)
{
if (auto * func = ast->as<ASTFunction>())
visit(*func, ast, data);
}
static bool needChildVisit(const ASTPtr & node, const ASTPtr &)
{
if (auto * func = node->as<ASTFunction>())
if (func->name == "equals")
return false;
return true;
}
private:
static void visit(const ASTFunction & func, const ASTPtr & ast, Data & data)
{
if (func.name == "and")
return; /// go into children
if (func.name == "equals")
{
ASTPtr left = func.arguments->children.at(0)->clone();
ASTPtr right = func.arguments->children.at(1)->clone();
addJoinKeys(ast, left, right, data);
return;
}
throwSyntaxException("Expected equals expression, got " + queryToString(ast) + ".");
}
static void getIdentifiers(const ASTPtr & ast, std::vector<const ASTIdentifier *> & out)
{
if (const auto * ident = ast->as<ASTIdentifier>())
{
if (IdentifierSemantic::getColumnName(*ident))
out.push_back(ident);
return;
}
for (const auto & child : ast->children)
getIdentifiers(child, out);
}
static void addJoinKeys(const ASTPtr & expr, ASTPtr left_ast, ASTPtr right_ast, Data & data)
{
std::vector<const ASTIdentifier *> left_identifiers;
std::vector<const ASTIdentifier *> right_identifiers;
getIdentifiers(left_ast, left_identifiers);
getIdentifiers(right_ast, right_identifiers);
size_t left_idents_table = getTableForIdentifiers(left_identifiers, data);
size_t right_idents_table = getTableForIdentifiers(right_identifiers, data);
if (left_idents_table && left_idents_table == right_idents_table)
{
auto left_name = queryToString(*left_identifiers[0]);
auto right_name = queryToString(*right_identifiers[0]);
throwSyntaxException("In expression " + queryToString(expr) + " columns " + left_name + " and " + right_name
+ " are from the same table but from different arguments of equal function.");
}
if (left_idents_table == 1 || right_idents_table == 2)
data.analyzed_join.addOnKeys(left_ast, right_ast);
else if (left_idents_table == 2 || right_idents_table == 1)
data.analyzed_join.addOnKeys(right_ast, left_ast);
else
throw Exception("Cannot detect left and right JOIN keys. JOIN ON section is ambiguous.",
ErrorCodes::AMBIGUOUS_COLUMN_NAME);
data.has_some = true;
}
static const ASTIdentifier * unrollAliases(const ASTIdentifier * identifier, const Aliases & aliases)
{
if (identifier->compound())
return identifier;
UInt32 max_attempts = 100;
for (auto it = aliases.find(identifier->name); it != aliases.end();)
{
const ASTIdentifier * parent = identifier;
identifier = it->second->as<ASTIdentifier>();
if (!identifier)
break; /// not a column alias
if (identifier == parent)
break; /// alias to itself with the same name: 'a as a'
if (identifier->compound())
break; /// not an alias. Break to prevent cycle through short names: 'a as b, t1.b as a'
it = aliases.find(identifier->name);
if (!max_attempts--)
throw Exception("Cannot unroll aliases for '" + identifier->name + "'", ErrorCodes::LOGICAL_ERROR);
}
return identifier;
}
/// @returns 1 if identifiers belongs to left table, 2 for right table and 0 if unknown. Throws on table mix.
/// Place detected identifier into identifiers[0] if any.
static size_t getTableForIdentifiers(std::vector<const ASTIdentifier *> & identifiers, const Data & data)
{
size_t table_number = 0;
for (auto & ident : identifiers)
{
const ASTIdentifier * identifier = unrollAliases(ident, data.aliases);
if (!identifier)
continue;
/// Column name could be cropped to a short form in TranslateQualifiedNamesVisitor.
/// In this case it saves membership in IdentifierSemantic.
size_t membership = IdentifierSemantic::getMembership(*identifier);
if (!membership)
{
const String & name = identifier->name;
bool in_left_table = data.source_columns.count(name);
bool in_right_table = data.joined_columns.count(name);
if (in_left_table && in_right_table)
throw Exception("Column '" + name + "' is ambiguous", ErrorCodes::AMBIGUOUS_COLUMN_NAME);
if (in_left_table)
membership = 1;
if (in_right_table)
membership = 2;
}
if (membership && table_number == 0)
{
table_number = membership;
std::swap(ident, identifiers[0]); /// move first detected identifier to the first position
}
if (membership && membership != table_number)
{
throw Exception("Invalid columns in JOIN ON section. Columns "
+ identifiers[0]->getAliasOrColumnName() + " and " + ident->getAliasOrColumnName()
+ " are from different tables.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
}
}
return table_number;
}
[[noreturn]] static void throwSyntaxException(const String & msg)
{
throw Exception("Invalid expression for JOIN ON. " + msg +
" Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) "
"[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]",
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
}
};
/// Parse JOIN ON expression and collect ASTs for joined columns.
using CollectJoinOnKeysVisitor = CollectJoinOnKeysMatcher::Visitor;
}

View File

@ -10,19 +10,19 @@ namespace DB
/// Visits AST tree in depth, call functions for nodes according to Matcher type data. /// Visits AST tree in depth, call functions for nodes according to Matcher type data.
/// You need to define Data, visit() and needChildVisit() in Matcher class. /// You need to define Data, visit() and needChildVisit() in Matcher class.
template <typename Matcher, bool _top_to_bottom> template <typename Matcher, bool _top_to_bottom, typename T>
class InDepthNodeVisitor class InDepthNodeVisitorTemplate
{ {
public: public:
using Data = typename Matcher::Data; using Data = typename Matcher::Data;
InDepthNodeVisitor(Data & data_, std::ostream * ostr_ = nullptr) InDepthNodeVisitorTemplate(Data & data_, std::ostream * ostr_ = nullptr)
: data(data_), : data(data_),
visit_depth(0), visit_depth(0),
ostr(ostr_) ostr(ostr_)
{} {}
void visit(ASTPtr & ast) void visit(T & ast)
{ {
DumpASTNode dump(*ast, ostr, visit_depth, typeid(Matcher).name()); DumpASTNode dump(*ast, ostr, visit_depth, typeid(Matcher).name());
@ -40,7 +40,7 @@ private:
size_t visit_depth; size_t visit_depth;
std::ostream * ostr; std::ostream * ostr;
void visitChildren(ASTPtr & ast) void visitChildren(T & ast)
{ {
for (auto & child : ast->children) for (auto & child : ast->children)
if (Matcher::needChildVisit(ast, child)) if (Matcher::needChildVisit(ast, child))
@ -48,6 +48,12 @@ private:
} }
}; };
template <typename Matcher, bool top_to_bottom>
using InDepthNodeVisitor = InDepthNodeVisitorTemplate<Matcher, top_to_bottom, ASTPtr>;
template <typename Matcher, bool top_to_bottom>
using ConstInDepthNodeVisitor = InDepthNodeVisitorTemplate<Matcher, top_to_bottom, const ASTPtr>;
/// Simple matcher for one node type without complex traversal logic. /// Simple matcher for one node type without complex traversal logic.
template <typename _Data, bool _visit_children = true> template <typename _Data, bool _visit_children = true>
class OneTypeMatcher class OneTypeMatcher

View File

@ -138,7 +138,10 @@ bool PredicateExpressionsOptimizer::allowPushDown(
const std::vector<IdentifierWithQualifier> & dependencies, const std::vector<IdentifierWithQualifier> & dependencies,
OptimizeKind & optimize_kind) OptimizeKind & optimize_kind)
{ {
if (!subquery || subquery->final() || subquery->limitBy() || subquery->limitLength() || subquery->with()) if (!subquery
|| (!settings.enable_optimize_predicate_expression_to_final_subquery && subquery->final())
|| subquery->limitBy() || subquery->limitLength()
|| subquery->with())
return false; return false;
else else
{ {

View File

@ -39,6 +39,7 @@ class PredicateExpressionsOptimizer
/// for PredicateExpressionsOptimizer /// for PredicateExpressionsOptimizer
const bool enable_optimize_predicate_expression; const bool enable_optimize_predicate_expression;
const bool enable_optimize_predicate_expression_to_final_subquery;
const bool join_use_nulls; const bool join_use_nulls;
template<typename T> template<typename T>
@ -47,6 +48,7 @@ class PredicateExpressionsOptimizer
max_expanded_ast_elements(settings.max_expanded_ast_elements), max_expanded_ast_elements(settings.max_expanded_ast_elements),
count_distinct_implementation(settings.count_distinct_implementation), count_distinct_implementation(settings.count_distinct_implementation),
enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression),
enable_optimize_predicate_expression_to_final_subquery(settings.enable_optimize_predicate_expression_to_final_subquery),
join_use_nulls(settings.join_use_nulls) join_use_nulls(settings.join_use_nulls)
{} {}
}; };

View File

@ -5,6 +5,7 @@
#include <Interpreters/QueryAliasesVisitor.h> #include <Interpreters/QueryAliasesVisitor.h>
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h> #include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/formatAST.h> #include <Parsers/formatAST.h>
#include <Parsers/ASTSubquery.h> #include <Parsers/ASTSubquery.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -41,12 +42,25 @@ void QueryAliasesMatcher::visit(ASTPtr & ast, Data & data)
{ {
if (auto * s = ast->as<ASTSubquery>()) if (auto * s = ast->as<ASTSubquery>())
visit(*s, ast, data); visit(*s, ast, data);
else if (auto * q = ast->as<ASTSelectQuery>())
visit(*q, ast, data);
else if (auto * aj = ast->as<ASTArrayJoin>()) else if (auto * aj = ast->as<ASTArrayJoin>())
visit(*aj, ast, data); visit(*aj, ast, data);
else else
visitOther(ast, data); visitOther(ast, data);
} }
void QueryAliasesMatcher::visit(const ASTSelectQuery & select, const ASTPtr &, Data &)
{
ASTPtr with = select.with();
if (!with)
return;
for (auto & child : with->children)
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(child.get()))
ast_with_alias->prefer_alias_to_column_name = true;
}
/// The top-level aliases in the ARRAY JOIN section have a special meaning, we will not add them /// The top-level aliases in the ARRAY JOIN section have a special meaning, we will not add them
/// (skip the expression list itself and its children). /// (skip the expression list itself and its children).
void QueryAliasesMatcher::visit(const ASTArrayJoin &, const ASTPtr & ast, Data & data) void QueryAliasesMatcher::visit(const ASTArrayJoin &, const ASTPtr & ast, Data & data)
@ -83,11 +97,12 @@ void QueryAliasesMatcher::visit(ASTSubquery & subquery, const ASTPtr & ast, Data
while (aliases.count(alias)); while (aliases.count(alias));
subquery.setAlias(alias); subquery.setAlias(alias);
subquery.prefer_alias_to_column_name = true;
aliases[alias] = ast; aliases[alias] = ast;
} }
else else
visitOther(ast, data); visitOther(ast, data);
subquery.prefer_alias_to_column_name = true;
} }
void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data) void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data)

View File

@ -6,7 +6,7 @@
namespace DB namespace DB
{ {
class ASTSelectWithUnionQuery; class ASTSelectQuery;
class ASTSubquery; class ASTSubquery;
struct ASTTableExpression; struct ASTTableExpression;
struct ASTArrayJoin; struct ASTArrayJoin;
@ -26,6 +26,7 @@ public:
static bool needChildVisit(ASTPtr & node, const ASTPtr & child); static bool needChildVisit(ASTPtr & node, const ASTPtr & child);
private: private:
static void visit(const ASTSelectQuery & select, const ASTPtr & ast, Data & data);
static void visit(ASTSubquery & subquery, const ASTPtr & ast, Data & data); static void visit(ASTSubquery & subquery, const ASTPtr & ast, Data & data);
static void visit(const ASTArrayJoin &, const ASTPtr & ast, Data & data); static void visit(const ASTArrayJoin &, const ASTPtr & ast, Data & data);
static void visitOther(const ASTPtr & ast, Data & data); static void visitOther(const ASTPtr & ast, Data & data);

View File

@ -10,6 +10,7 @@
#include <Interpreters/QueryNormalizer.h> #include <Interpreters/QueryNormalizer.h>
#include <Interpreters/ExecuteScalarSubqueriesVisitor.h> #include <Interpreters/ExecuteScalarSubqueriesVisitor.h>
#include <Interpreters/PredicateExpressionsOptimizer.h> #include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/CollectJoinOnKeysVisitor.h>
#include <Interpreters/ExternalDictionaries.h> #include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h> #include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
@ -21,7 +22,6 @@
#include <Parsers/ASTTablesInSelectQuery.h> #include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ParserTablesInSelectQuery.h> #include <Parsers/ParserTablesInSelectQuery.h>
#include <Parsers/parseQuery.h> #include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <DataTypes/NestedUtils.h> #include <DataTypes/NestedUtils.h>
@ -481,126 +481,11 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
} }
} }
[[noreturn]] static void throwSyntaxException(const String & msg)
{
throw Exception("Invalid expression for JOIN ON. " + msg + " Supported syntax: JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) "
"[AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]", ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
};
/// Parse JOIN ON expression and collect ASTs for joined columns.
void collectJoinedColumnsFromJoinOnExpr(AnalyzedJoin & analyzed_join, const ASTTableJoin & table_join)
{
if (!table_join.on_expression)
return;
/// Stores examples of columns which are only from one table.
struct TableBelonging
{
const ASTIdentifier * example_only_from_left = nullptr;
const ASTIdentifier * example_only_from_right = nullptr;
};
/// Check all identifiers in ast and decide their possible table belonging.
/// Throws if there are two identifiers definitely from different tables.
std::function<TableBelonging(const ASTPtr &)> get_table_belonging;
get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging
{
if (IdentifierSemantic::getColumnName(ast))
{
const auto * identifier = ast->as<ASTIdentifier>();
/// It's set in TranslateQualifiedNamesVisitor
size_t membership = IdentifierSemantic::getMembership(*identifier);
switch (membership)
{
case 1: return {identifier, nullptr};
case 2: return {nullptr, identifier};
default:
break;
}
return {};
}
TableBelonging table_belonging;
for (const auto & child : ast->children)
{
auto children_belonging = get_table_belonging(child);
if (!table_belonging.example_only_from_left)
table_belonging.example_only_from_left = children_belonging.example_only_from_left;
if (!table_belonging.example_only_from_right)
table_belonging.example_only_from_right = children_belonging.example_only_from_right;
}
if (table_belonging.example_only_from_left && table_belonging.example_only_from_right)
throw Exception("Invalid columns in JOIN ON section. Columns "
+ table_belonging.example_only_from_left->getAliasOrColumnName() + " and "
+ table_belonging.example_only_from_right->getAliasOrColumnName()
+ " are from different tables.", ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
return table_belonging;
};
/// For equal expression find out corresponding table for each part, translate qualified names and add asts to join keys.
auto add_columns_from_equals_expr = [&](const ASTPtr & expr)
{
const auto * func_equals = expr->as<ASTFunction>();
if (!func_equals || func_equals->name != "equals")
throwSyntaxException("Expected equals expression, got " + queryToString(expr) + ".");
ASTPtr left_ast = func_equals->arguments->children.at(0)->clone();
ASTPtr right_ast = func_equals->arguments->children.at(1)->clone();
auto left_table_belonging = get_table_belonging(left_ast);
auto right_table_belonging = get_table_belonging(right_ast);
bool can_be_left_part_from_left_table = left_table_belonging.example_only_from_right == nullptr;
bool can_be_left_part_from_right_table = left_table_belonging.example_only_from_left == nullptr;
bool can_be_right_part_from_left_table = right_table_belonging.example_only_from_right == nullptr;
bool can_be_right_part_from_right_table = right_table_belonging.example_only_from_left == nullptr;
/// Default variant when all identifiers may be from any table.
if (can_be_left_part_from_left_table && can_be_right_part_from_right_table)
analyzed_join.addOnKeys(left_ast, right_ast);
else if (can_be_left_part_from_right_table && can_be_right_part_from_left_table)
analyzed_join.addOnKeys(right_ast, left_ast);
else
{
auto * left_example = left_table_belonging.example_only_from_left ?
left_table_belonging.example_only_from_left :
left_table_belonging.example_only_from_right;
auto * right_example = right_table_belonging.example_only_from_left ?
right_table_belonging.example_only_from_left :
right_table_belonging.example_only_from_right;
auto left_name = queryToString(*left_example);
auto right_name = queryToString(*right_example);
auto expr_name = queryToString(expr);
throwSyntaxException("In expression " + expr_name + " columns " + left_name + " and " + right_name
+ " are from the same table but from different arguments of equal function.");
}
};
const auto * func = table_join.on_expression->as<ASTFunction>();
if (func && func->name == "and")
{
for (const auto & expr : func->arguments->children)
add_columns_from_equals_expr(expr);
}
else
add_columns_from_equals_expr(table_join.on_expression);
}
/// Find the columns that are obtained by JOIN. /// Find the columns that are obtained by JOIN.
void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, const NameSet & source_columns,
const NameSet & source_columns, const String & current_database, bool join_use_nulls) const Aliases & aliases, const String & current_database, bool join_use_nulls)
{ {
const ASTTablesInSelectQueryElement * node = select_query.join(); const ASTTablesInSelectQueryElement * node = select_query.join();
if (!node) if (!node)
return; return;
@ -619,7 +504,17 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s
name = joined_table_name.getQualifiedNamePrefix() + name; name = joined_table_name.getQualifiedNamePrefix() + name;
} }
else if (table_join.on_expression) else if (table_join.on_expression)
collectJoinedColumnsFromJoinOnExpr(analyzed_join, table_join); {
NameSet joined_columns;
for (const auto & col : analyzed_join.columns_from_joined_table)
joined_columns.insert(col.original_name);
CollectJoinOnKeysVisitor::Data data{analyzed_join, source_columns, joined_columns, aliases};
CollectJoinOnKeysVisitor(data).visit(table_join.on_expression);
if (!data.has_some)
throw Exception("Cannot get JOIN keys from JOIN ON section: " + queryToString(table_join.on_expression),
ErrorCodes::INVALID_JOIN_ON_EXPRESSION);
}
bool make_nullable = join_use_nulls && isLeftOrFull(table_join.kind); bool make_nullable = join_use_nulls && isLeftOrFull(table_join.kind);
@ -771,7 +666,8 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
/// Push the predicate expression down to the subqueries. /// Push the predicate expression down to the subqueries.
result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, context.getCurrentDatabase(), settings.join_use_nulls); collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases,
context.getCurrentDatabase(), settings.join_use_nulls);
} }
return std::make_shared<const SyntaxAnalyzerResult>(result); return std::make_shared<const SyntaxAnalyzerResult>(result);

View File

@ -41,8 +41,6 @@ void ThreadStatus::attachQueryContext(Context & query_context_)
if (!thread_group->global_context) if (!thread_group->global_context)
thread_group->global_context = global_context; thread_group->global_context = global_context;
} }
initQueryProfiler();
} }
void CurrentThread::defaultThreadDeleter() void CurrentThread::defaultThreadDeleter()
@ -124,6 +122,7 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool
#endif #endif
initPerformanceCounters(); initPerformanceCounters();
initQueryProfiler();
thread_state = ThreadState::AttachedToQuery; thread_state = ThreadState::AttachedToQuery;
} }
@ -155,7 +154,7 @@ void ThreadStatus::finalizePerformanceCounters()
void ThreadStatus::initQueryProfiler() void ThreadStatus::initQueryProfiler()
{ {
/// query profilers are useless without trace collector /// query profilers are useless without trace collector
if (!global_context->hasTraceCollector()) if (!global_context || !global_context->hasTraceCollector())
return; return;
const auto & settings = query_context->getSettingsRef(); const auto & settings = query_context->getSettingsRef();
@ -163,14 +162,12 @@ void ThreadStatus::initQueryProfiler()
if (settings.query_profiler_real_time_period_ns > 0) if (settings.query_profiler_real_time_period_ns > 0)
query_profiler_real = std::make_unique<QueryProfilerReal>( query_profiler_real = std::make_unique<QueryProfilerReal>(
/* thread_id */ os_thread_id, /* thread_id */ os_thread_id,
/* period */ static_cast<UInt32>(settings.query_profiler_real_time_period_ns) /* period */ static_cast<UInt32>(settings.query_profiler_real_time_period_ns));
);
if (settings.query_profiler_cpu_time_period_ns > 0) if (settings.query_profiler_cpu_time_period_ns > 0)
query_profiler_cpu = std::make_unique<QueryProfilerCpu>( query_profiler_cpu = std::make_unique<QueryProfilerCpu>(
/* thread_id */ os_thread_id, /* thread_id */ os_thread_id,
/* period */ static_cast<UInt32>(settings.query_profiler_cpu_time_period_ns) /* period */ static_cast<UInt32>(settings.query_profiler_cpu_time_period_ns));
);
} }
void ThreadStatus::finalizeQueryProfiler() void ThreadStatus::finalizeQueryProfiler()

View File

@ -4,6 +4,8 @@
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeDate.h> #include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <Common/ClickHouseRevision.h>
using namespace DB; using namespace DB;
@ -20,13 +22,15 @@ Block TraceLogElement::createBlock()
{ {
{std::make_shared<DataTypeDate>(), "event_date"}, {std::make_shared<DataTypeDate>(), "event_date"},
{std::make_shared<DataTypeDateTime>(), "event_time"}, {std::make_shared<DataTypeDateTime>(), "event_time"},
{std::make_shared<DataTypeUInt32>(), "revision"},
{std::make_shared<TimerDataType>(timer_values), "timer_type"}, {std::make_shared<TimerDataType>(timer_values), "timer_type"},
{std::make_shared<DataTypeUInt32>(), "thread_number"},
{std::make_shared<DataTypeString>(), "query_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"}
}; };
} }
void TraceLogElement::appendToBlock(Block &block) const void TraceLogElement::appendToBlock(Block & block) const
{ {
MutableColumns columns = block.mutateColumns(); MutableColumns columns = block.mutateColumns();
@ -34,7 +38,9 @@ void TraceLogElement::appendToBlock(Block &block) const
columns[i++]->insert(DateLUT::instance().toDayNum(event_time)); columns[i++]->insert(DateLUT::instance().toDayNum(event_time));
columns[i++]->insert(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>(timer_type));
columns[i++]->insert(thread_number);
columns[i++]->insertData(query_id.data(), query_id.size()); columns[i++]->insertData(query_id.data(), query_id.size());
columns[i++]->insert(trace); columns[i++]->insert(trace);

View File

@ -15,7 +15,8 @@ struct TraceLogElement
static const TimerDataType::Values timer_values; static const TimerDataType::Values timer_values;
time_t event_time{}; time_t event_time{};
TimerType timer_type; TimerType timer_type{};
UInt32 thread_number{};
String query_id{}; String query_id{};
Array trace{}; Array trace{};

View File

@ -24,7 +24,7 @@ public:
struct Data struct Data
{ {
NameSet source_columns; const NameSet source_columns;
const std::vector<TableWithColumnNames> & tables; const std::vector<TableWithColumnNames> & tables;
std::unordered_set<String> join_using_columns; std::unordered_set<String> join_using_columns;
bool has_columns; bool has_columns;

View File

@ -1337,7 +1337,6 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(node.get())) if (auto * ast_with_alias = dynamic_cast<ASTWithAlias *>(node.get()))
{ {
getIdentifierName(alias_node, ast_with_alias->alias); getIdentifierName(alias_node, ast_with_alias->alias);
ast_with_alias->prefer_alias_to_column_name = prefer_alias_to_column_name;
} }
else else
{ {

View File

@ -274,13 +274,12 @@ protected:
class ParserWithOptionalAlias : public IParserBase class ParserWithOptionalAlias : public IParserBase
{ {
public: public:
ParserWithOptionalAlias(ParserPtr && elem_parser_, bool allow_alias_without_as_keyword_, bool prefer_alias_to_column_name_ = false) ParserWithOptionalAlias(ParserPtr && elem_parser_, bool allow_alias_without_as_keyword_)
: elem_parser(std::move(elem_parser_)), allow_alias_without_as_keyword(allow_alias_without_as_keyword_), : elem_parser(std::move(elem_parser_)), allow_alias_without_as_keyword(allow_alias_without_as_keyword_)
prefer_alias_to_column_name(prefer_alias_to_column_name_) {} {}
protected: protected:
ParserPtr elem_parser; ParserPtr elem_parser;
bool allow_alias_without_as_keyword; bool allow_alias_without_as_keyword;
bool prefer_alias_to_column_name;
const char * getName() const { return "element of expression with optional alias"; } const char * getName() const { return "element of expression with optional alias"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);

View File

@ -522,9 +522,9 @@ bool ParserTupleElementExpression::parseImpl(Pos & pos, ASTPtr & node, Expected
} }
ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool prefer_alias_to_column_name) ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword)
: impl(std::make_unique<ParserWithOptionalAlias>(std::make_unique<ParserExpression>(), : impl(std::make_unique<ParserWithOptionalAlias>(std::make_unique<ParserExpression>(),
allow_alias_without_as_keyword, prefer_alias_to_column_name)) allow_alias_without_as_keyword))
{ {
} }
@ -532,7 +532,7 @@ ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_
bool ParserExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserExpressionList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{ {
return ParserList( return ParserList(
std::make_unique<ParserExpressionWithOptionalAlias>(allow_alias_without_as_keyword, prefer_alias_to_column_name), std::make_unique<ParserExpressionWithOptionalAlias>(allow_alias_without_as_keyword),
std::make_unique<ParserToken>(TokenType::Comma)) std::make_unique<ParserToken>(TokenType::Comma))
.parse(pos, node, expected); .parse(pos, node, expected);
} }

View File

@ -322,7 +322,7 @@ using ParserExpression = ParserLambdaExpression;
class ParserExpressionWithOptionalAlias : public IParserBase class ParserExpressionWithOptionalAlias : public IParserBase
{ {
public: public:
ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool prefer_alias_to_column_name_ = false); ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword);
protected: protected:
ParserPtr impl; ParserPtr impl;
@ -339,12 +339,11 @@ protected:
class ParserExpressionList : public IParserBase class ParserExpressionList : public IParserBase
{ {
public: public:
ParserExpressionList(bool allow_alias_without_as_keyword_, bool prefer_alias_to_column_name_ = false) ParserExpressionList(bool allow_alias_without_as_keyword_)
: allow_alias_without_as_keyword(allow_alias_without_as_keyword_), prefer_alias_to_column_name(prefer_alias_to_column_name_) {} : allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {}
protected: protected:
bool allow_alias_without_as_keyword; bool allow_alias_without_as_keyword;
bool prefer_alias_to_column_name;
const char * getName() const { return "list of expressions"; } const char * getName() const { return "list of expressions"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
@ -354,8 +353,8 @@ protected:
class ParserNotEmptyExpressionList : public IParserBase class ParserNotEmptyExpressionList : public IParserBase
{ {
public: public:
ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword, bool prefer_alias_to_column_name = false) ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword)
: nested_parser(allow_alias_without_as_keyword, prefer_alias_to_column_name) {} : nested_parser(allow_alias_without_as_keyword) {}
private: private:
ParserExpressionList nested_parser; ParserExpressionList nested_parser;
protected: protected:

View File

@ -44,7 +44,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_offset("OFFSET"); ParserKeyword s_offset("OFFSET");
ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list(false);
ParserNotEmptyExpressionList exp_list_for_with_clause(false, true); /// Set prefer_alias_to_column_name for each alias. ParserNotEmptyExpressionList exp_list_for_with_clause(false);
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.
ParserExpressionWithOptionalAlias exp_elem(false); ParserExpressionWithOptionalAlias exp_elem(false);
ParserOrderByExpressionList order_list; ParserOrderByExpressionList order_list;

View File

@ -43,7 +43,7 @@ bool ProtobufRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
read_columns[column_index] = true; read_columns[column_index] = true;
allow_add_row = false; allow_add_row = false;
} }
} while (reader.maybeCanReadValue()); } while (reader.canReadMoreValues());
} }
// Fill non-visited columns with the default values. // Fill non-visited columns with the default values.
@ -62,7 +62,7 @@ bool ProtobufRowInputFormat::allowSyncAfterError() const
void ProtobufRowInputFormat::syncAfterError() void ProtobufRowInputFormat::syncAfterError()
{ {
reader.endMessage(); reader.endMessage(true);
} }

View File

@ -28,7 +28,10 @@ KafkaBlockInputStream::~KafkaBlockInputStream()
return; return;
if (broken) if (broken)
{
buffer->subBufferAs<ReadBufferFromKafkaConsumer>()->unsubscribe(); buffer->subBufferAs<ReadBufferFromKafkaConsumer>()->unsubscribe();
buffer->reset();
}
storage.pushBuffer(buffer); storage.pushBuffer(buffer);
} }

View File

@ -90,6 +90,11 @@ void ReadBufferFromKafkaConsumer::subscribe(const Names & topics)
void ReadBufferFromKafkaConsumer::unsubscribe() void ReadBufferFromKafkaConsumer::unsubscribe()
{ {
LOG_TRACE(log, "Re-joining claimed consumer after failure"); LOG_TRACE(log, "Re-joining claimed consumer after failure");
messages.clear();
current = messages.begin();
BufferBase::set(nullptr, 0, 0);
consumer->unsubscribe(); consumer->unsubscribe();
} }

View File

@ -689,6 +689,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
MonotonicFunctionsChain chain; MonotonicFunctionsChain chain;
std::string func_name = func->name; std::string func_name = func->name;
if (atom_map.find(func_name) == std::end(atom_map))
return false;
if (args.size() == 1) if (args.size() == 1)
{ {
if (!(isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))) if (!(isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain)))
@ -775,8 +778,6 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
return false; return false;
const auto atom_it = atom_map.find(func_name); const auto atom_it = atom_map.find(func_name);
if (atom_it == std::end(atom_map))
return false;
out.key_column = key_column_num; out.key_column = key_column_num;
out.monotonic_functions_chain = std::move(chain); out.monotonic_functions_chain = std::move(chain);

View File

@ -27,7 +27,8 @@ void MergeTreeBlockOutputStream::write(const Block & block)
PartLog::addNewPart(storage.global_context, part, watch.elapsed()); PartLog::addNewPart(storage.global_context, part, watch.elapsed());
/// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'. /// Initiate async merge - it will be done if it's good time for merge and if there are space in 'background_pool'.
storage.background_task_handle->wake(); if (storage.background_task_handle)
storage.background_task_handle->wake();
} }
} }

View File

@ -381,18 +381,18 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
} }
rows_count += rows; rows_count += rows;
{ {
/// Creating block for update /// Creating block for update
Block indices_update_block(skip_indexes_columns); Block indices_update_block(skip_indexes_columns);
size_t skip_index_current_mark = 0;
/// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn
for (size_t i = 0; i < storage.skip_indices.size(); ++i) for (size_t i = 0; i < storage.skip_indices.size(); ++i)
{ {
const auto index = storage.skip_indices[i]; const auto index = storage.skip_indices[i];
auto & stream = *skip_indices_streams[i]; auto & stream = *skip_indices_streams[i];
size_t prev_pos = 0; size_t prev_pos = 0;
skip_index_current_mark = skip_index_mark;
size_t skip_index_current_mark = 0;
while (prev_pos < rows) while (prev_pos < rows)
{ {
UInt64 limit = 0; UInt64 limit = 0;
@ -417,6 +417,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
/// to be compatible with normal .mrk2 file format /// to be compatible with normal .mrk2 file format
if (storage.canUseAdaptiveGranularity()) if (storage.canUseAdaptiveGranularity())
writeIntBinary(1UL, stream.marks); writeIntBinary(1UL, stream.marks);
++skip_index_current_mark;
} }
} }
@ -435,9 +437,9 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm
} }
} }
prev_pos = pos; prev_pos = pos;
++skip_index_current_mark;
} }
} }
skip_index_mark = skip_index_current_mark;
} }
{ {

View File

@ -68,6 +68,7 @@ private:
String part_path; String part_path;
size_t rows_count = 0; size_t rows_count = 0;
size_t skip_index_mark = 0;
std::unique_ptr<WriteBufferFromFile> index_file_stream; std::unique_ptr<WriteBufferFromFile> index_file_stream;
std::unique_ptr<HashingWriteBuffer> index_stream; std::unique_ptr<HashingWriteBuffer> index_stream;

View File

@ -6,6 +6,8 @@
#include <DataStreams/OneBlockInputStream.h> #include <DataStreams/OneBlockInputStream.h>
#include <DataStreams/ConcatBlockInputStream.h> #include <DataStreams/ConcatBlockInputStream.h>
#include <DataStreams/materializeBlock.h> #include <DataStreams/materializeBlock.h>
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/FilterBlockInputStream.h>
#include <Storages/StorageMerge.h> #include <Storages/StorageMerge.h>
#include <Storages/StorageFactory.h> #include <Storages/StorageFactory.h>
#include <Storages/VirtualColumnUtils.h> #include <Storages/VirtualColumnUtils.h>
@ -23,8 +25,6 @@
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Core/SettingsCommon.h> #include <Core/SettingsCommon.h>
#include <DataStreams/MaterializingBlockInputStream.h>
#include <DataStreams/FilterBlockInputStream.h>
#include <ext/range.h> #include <ext/range.h>
#include <algorithm> #include <algorithm>
#include <Parsers/ASTFunction.h> #include <Parsers/ASTFunction.h>

View File

@ -67,5 +67,5 @@ sudo -u clickhouse UBSAN_OPTIONS='print_stacktrace=1' ./clickhouse-ubsan server
# How to use Memory Sanitizer # How to use Memory Sanitizer
``` ```
CC=clang-8 CXX=clang++-8 cmake -D ENABLE_HDFS=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_POCO_ODBC=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0 -D ENABLE_EMBEDDED_COMPILER=0 -D USE_INTERNAL_CAPNP_LIBRARY=0 -D USE_INTERNAL_SSL_LIBRARY=0 -D USE_SIMDJSON=0 -DENABLE_READLINE=0 -D SANITIZE=memory .. CC=clang-8 CXX=clang++-8 cmake -D ENABLE_HDFS=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_POCO_ODBC=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0 -D ENABLE_EMBEDDED_COMPILER=0 -D USE_INTERNAL_CAPNP_LIBRARY=0 -D USE_SIMDJSON=0 -DENABLE_READLINE=0 -D SANITIZE=memory ..
``` ```

View File

@ -0,0 +1,4 @@
<remote_servers>
</remote_servers>

View File

@ -0,0 +1,415 @@
<?xml version="1.0"?>
<!--
NOTE: User and query level settings are set up in "users.xml" file.
-->
<yandex>
<logger>
<!-- Possible levels: https://github.com/pocoproject/poco/blob/develop/Foundation/include/Poco/Logger.h#L105 -->
<level>trace</level>
<log>/var/log/clickhouse-server/clickhouse-server.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<!-- <console>1</console> --> <!-- Default behavior is autodetection (log to console if not daemon mode and is tty) -->
</logger>
<!--display_name>production</display_name--> <!-- It is the name that will be shown in the client -->
<http_port>8123</http_port>
<tcp_port>9000</tcp_port>
<!-- For HTTPS and SSL over native protocol. -->
<!--
<https_port>8443</https_port>
<tcp_port_secure>9440</tcp_port_secure>
-->
<!-- Used with https_port and tcp_port_secure. Full ssl options list: https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h#L71 -->
<openSSL>
<server> <!-- Used for https server AND secure tcp port -->
<!-- openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt -->
<certificateFile>/etc/clickhouse-server/server.crt</certificateFile>
<privateKeyFile>/etc/clickhouse-server/server.key</privateKeyFile>
<!-- openssl dhparam -out /etc/clickhouse-server/dhparam.pem 4096 -->
<dhParamsFile>/etc/clickhouse-server/dhparam.pem</dhParamsFile>
<verificationMode>none</verificationMode>
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
</server>
<client> <!-- Used for connecting to https dictionary source -->
<loadDefaultCAFile>true</loadDefaultCAFile>
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<!-- Use for self-signed: <verificationMode>none</verificationMode> -->
<invalidCertificateHandler>
<!-- Use for self-signed: <name>AcceptCertificateHandler</name> -->
<name>RejectCertificateHandler</name>
</invalidCertificateHandler>
</client>
</openSSL>
<!-- Default root page on http[s] server. For example load UI from https://tabix.io/ when opening http://localhost:8123 -->
<!--
<http_server_default_response><![CDATA[<html ng-app="SMI2"><head><base href="http://ui.tabix.io/"></head><body><div ui-view="" class="content-ui"></div><script src="http://loader.tabix.io/master.js"></script></body></html>]]></http_server_default_response>
-->
<!-- Port for communication between replicas. Used for data exchange. -->
<interserver_http_port>9009</interserver_http_port>
<!-- Hostname that is used by other replicas to request this server.
If not specified, than it is determined analoguous to 'hostname -f' command.
This setting could be used to switch replication to another network interface.
-->
<!--
<interserver_http_host>example.yandex.ru</interserver_http_host>
-->
<!-- Listen specified host. use :: (wildcard IPv6 address), if you want to accept connections both with IPv4 and IPv6 from everywhere. -->
<!-- <listen_host>::</listen_host> -->
<!-- Same for hosts with disabled ipv6: -->
<!-- <listen_host>0.0.0.0</listen_host> -->
<!-- Default values - try listen localhost on ipv4 and ipv6: -->
<!--
<listen_host>::1</listen_host>
<listen_host>127.0.0.1</listen_host>
-->
<!-- Don't exit if ipv6 or ipv4 unavailable, but listen_host with this protocol specified -->
<!-- <listen_try>0</listen_try> -->
<!-- Allow listen on same address:port -->
<!-- <listen_reuse_port>0</listen_reuse_port> -->
<!-- <listen_backlog>64</listen_backlog> -->
<max_connections>4096</max_connections>
<keep_alive_timeout>3</keep_alive_timeout>
<!-- Maximum number of concurrent queries. -->
<max_concurrent_queries>100</max_concurrent_queries>
<!-- Set limit on number of open files (default: maximum). This setting makes sense on Mac OS X because getrlimit() fails to retrieve
correct maximum value. -->
<!-- <max_open_files>262144</max_open_files> -->
<!-- Size of cache of uncompressed blocks of data, used in tables of MergeTree family.
In bytes. Cache is single for server. Memory is allocated only on demand.
Cache is used when 'use_uncompressed_cache' user setting turned on (off by default).
Uncompressed cache is advantageous only for very short queries and in rare cases.
-->
<uncompressed_cache_size>8589934592</uncompressed_cache_size>
<!-- Approximate size of mark cache, used in tables of MergeTree family.
In bytes. Cache is single for server. Memory is allocated only on demand.
You should not lower this value.
-->
<mark_cache_size>5368709120</mark_cache_size>
<!-- Path to data directory, with trailing slash. -->
<path>/var/lib/clickhouse/</path>
<!-- Path to temporary data for processing hard queries. -->
<tmp_path>/var/lib/clickhouse/tmp/</tmp_path>
<!-- Directory with user provided files that are accessible by 'file' table function. -->
<user_files_path>/var/lib/clickhouse/user_files/</user_files_path>
<!-- Path to configuration file with users, access rights, profiles of settings, quotas. -->
<users_config>users.xml</users_config>
<!-- Default profile of settings. -->
<default_profile>default</default_profile>
<!-- System profile of settings. This settings are used by internal processes (Buffer storage, Distibuted DDL worker and so on). -->
<!-- <system_profile>default</system_profile> -->
<!-- Default database. -->
<default_database>default</default_database>
<!-- Server time zone could be set here.
Time zone is used when converting between String and DateTime types,
when printing DateTime in text formats and parsing DateTime from text,
it is used in date and time related functions, if specific time zone was not passed as an argument.
Time zone is specified as identifier from IANA time zone database, like UTC or Africa/Abidjan.
If not specified, system time zone at server startup is used.
Please note, that server could display time zone alias instead of specified name.
Example: W-SU is an alias for Europe/Moscow and Zulu is an alias for UTC.
-->
<!-- <timezone>Europe/Moscow</timezone> -->
<!-- You can specify umask here (see "man umask"). Server will apply it on startup.
Number is always parsed as octal. Default umask is 027 (other users cannot read logs, data files, etc; group can only read).
-->
<!-- <umask>022</umask> -->
<!-- Perform mlockall after startup to lower first queries latency
and to prevent clickhouse executable from being paged out under high IO load.
Enabling this option is recommended but will lead to increased startup time for up to a few seconds.
-->
<mlock_executable>false</mlock_executable>
<!-- Configuration of clusters that could be used in Distributed tables.
https://clickhouse.yandex/docs/en/table_engines/distributed/
-->
<remote_servers incl="clickhouse_remote_servers" >
<!-- Test only shard config for testing distributed storage -->
<test_shard_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_shard_localhost>
<test_cluster_two_shards_localhost>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
</test_cluster_two_shards_localhost>
<test_shard_localhost_secure>
<shard>
<replica>
<host>localhost</host>
<port>9440</port>
<secure>1</secure>
</replica>
</shard>
</test_shard_localhost_secure>
<test_unavailable_shard>
<shard>
<replica>
<host>localhost</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>localhost</host>
<port>1</port>
</replica>
</shard>
</test_unavailable_shard>
</remote_servers>
<!-- If element has 'incl' attribute, then for it's value will be used corresponding substitution from another file.
By default, path to file with substitutions is /etc/metrika.xml. It could be changed in config in 'include_from' element.
Values for substitutions are specified in /yandex/name_of_substitution elements in that file.
-->
<!-- ZooKeeper is used to store metadata about replicas, when using Replicated tables.
Optional. If you don't use replicated tables, you could omit that.
See https://clickhouse.yandex/docs/en/table_engines/replication/
-->
<zookeeper incl="zookeeper-servers" optional="true" />
<!-- Substitutions for parameters of replicated tables.
Optional. If you don't use replicated tables, you could omit that.
See https://clickhouse.yandex/docs/en/table_engines/replication/#creating-replicated-tables
-->
<macros incl="macros" optional="true" />
<!-- Reloading interval for embedded dictionaries, in seconds. Default: 3600. -->
<builtin_dictionaries_reload_interval>3600</builtin_dictionaries_reload_interval>
<!-- Maximum session timeout, in seconds. Default: 3600. -->
<max_session_timeout>3600</max_session_timeout>
<!-- Default session timeout, in seconds. Default: 60. -->
<default_session_timeout>60</default_session_timeout>
<!-- Sending data to Graphite for monitoring. Several sections can be defined. -->
<!--
interval - send every X second
root_path - prefix for keys
hostname_in_path - append hostname to root_path (default = true)
metrics - send data from table system.metrics
events - send data from table system.events
asynchronous_metrics - send data from table system.asynchronous_metrics
-->
<!--
<graphite>
<host>localhost</host>
<port>42000</port>
<timeout>0.1</timeout>
<interval>60</interval>
<root_path>one_min</root_path>
<hostname_in_path>true</hostname_in_path>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>true</asynchronous_metrics>
</graphite>
<graphite>
<host>localhost</host>
<port>42000</port>
<timeout>0.1</timeout>
<interval>1</interval>
<root_path>one_sec</root_path>
<metrics>true</metrics>
<events>true</events>
<asynchronous_metrics>false</asynchronous_metrics>
</graphite>
-->
<!-- Query log. Used only for queries with setting log_queries = 1. -->
<query_log>
<!-- What table to insert data. If table is not exist, it will be created.
When query log structure is changed after system update,
then old table will be renamed and new table will be created automatically.
-->
<database>system</database>
<table>query_log</table>
<!--
PARTITION BY expr https://clickhouse.yandex/docs/en/table_engines/custom_partitioning_key/
Example:
event_date
toMonday(event_date)
toYYYYMM(event_date)
toStartOfHour(event_time)
-->
<partition_by>toYYYYMM(event_date)</partition_by>
<!-- Interval of flushing data. -->
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_log>
<!-- Query thread log. Has information about all threads participated in query execution.
Used only for queries with setting log_query_threads = 1. -->
<query_thread_log>
<database>system</database>
<table>query_thread_log</table>
<partition_by>toYYYYMM(event_date)</partition_by>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</query_thread_log>
<!-- Uncomment if use part log.
Part log contains information about all actions with parts in MergeTree tables (creation, deletion, merges, downloads).
<part_log>
<database>system</database>
<table>part_log</table>
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
</part_log>
-->
<!-- Parameters for embedded dictionaries, used in Yandex.Metrica.
See https://clickhouse.yandex/docs/en/dicts/internal_dicts/
-->
<!-- Path to file with region hierarchy. -->
<!-- <path_to_regions_hierarchy_file>/opt/geo/regions_hierarchy.txt</path_to_regions_hierarchy_file> -->
<!-- Path to directory with files containing names of regions -->
<!-- <path_to_regions_names_files>/opt/geo/</path_to_regions_names_files> -->
<!-- Configuration of external dictionaries. See:
https://clickhouse.yandex/docs/en/dicts/external_dicts/
-->
<dictionaries_config>*_dictionary.xml</dictionaries_config>
<!-- Uncomment if you want data to be compressed 30-100% better.
Don't do that if you just started using ClickHouse.
-->
<compression incl="clickhouse_compression">
<!--
<!- - Set of variants. Checked in order. Last matching case wins. If nothing matches, lz4 will be used. - ->
<case>
<!- - Conditions. All must be satisfied. Some conditions may be omitted. - ->
<min_part_size>10000000000</min_part_size> <!- - Min part size in bytes. - ->
<min_part_size_ratio>0.01</min_part_size_ratio> <!- - Min size of part relative to whole table size. - ->
<!- - What compression method to use. - ->
<method>zstd</method>
</case>
-->
</compression>
<!-- Allow to execute distributed DDL queries (CREATE, DROP, ALTER, RENAME) on cluster.
Works only if ZooKeeper is enabled. Comment it if such functionality isn't required. -->
<distributed_ddl>
<!-- Path in ZooKeeper to queue with DDL queries -->
<path>/clickhouse/task_queue/ddl</path>
<!-- Settings from this profile will be used to execute DDL queries -->
<!-- <profile>default</profile> -->
</distributed_ddl>
<!-- Settings to fine tune MergeTree tables. See documentation in source code, in MergeTreeSettings.h -->
<!--
<merge_tree>
<max_suspicious_broken_parts>5</max_suspicious_broken_parts>
</merge_tree>
-->
<!-- Protection from accidental DROP.
If size of a MergeTree table is greater than max_table_size_to_drop (in bytes) than table could not be dropped with any DROP query.
If you want do delete one table and don't want to restart clickhouse-server, you could create special file <clickhouse-path>/flags/force_drop_table and make DROP once.
By default max_table_size_to_drop is 50GB; max_table_size_to_drop=0 allows to DROP any tables.
The same for max_partition_size_to_drop.
Uncomment to disable protection.
-->
<!-- <max_table_size_to_drop>0</max_table_size_to_drop> -->
<!-- <max_partition_size_to_drop>0</max_partition_size_to_drop> -->
<!-- Example of parameters for GraphiteMergeTree table engine -->
<graphite_rollup_example>
<pattern>
<regexp>click_cost</regexp>
<function>any</function>
<retention>
<age>0</age>
<precision>3600</precision>
</retention>
<retention>
<age>86400</age>
<precision>60</precision>
</retention>
</pattern>
<default>
<function>max</function>
<retention>
<age>0</age>
<precision>60</precision>
</retention>
<retention>
<age>3600</age>
<precision>300</precision>
</retention>
<retention>
<age>86400</age>
<precision>3600</precision>
</retention>
</default>
</graphite_rollup_example>
<!-- Directory in <clickhouse-path> containing schema files for various input formats.
The directory will be created if it doesn't exist.
-->
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
<!-- Uncomment to disable ClickHouse internal DNS caching. -->
<!-- <disable_internal_dns_cache>1</disable_internal_dns_cache> -->
</yandex>

View File

@ -0,0 +1,130 @@
<?xml version="1.0"?>
<yandex>
<!-- Profiles of settings. -->
<profiles>
<!-- Default settings. -->
<default>
<!-- Maximum memory usage for processing single query, in bytes. -->
<max_memory_usage>10000000000</max_memory_usage>
<!-- Use cache of uncompressed blocks of data. Meaningful only for processing many of very short queries. -->
<use_uncompressed_cache>0</use_uncompressed_cache>
<!-- How to choose between replicas during distributed query processing.
random - choose random replica from set of replicas with minimum number of errors
nearest_hostname - from set of replicas with minimum number of errors, choose replica
with minimum number of different symbols between replica's hostname and local hostname
(Hamming distance).
in_order - first live replica is chosen in specified order.
first_or_random - if first replica one has higher number of errors, pick a random one from replicas with minimum number of errors.
-->
<load_balancing>random</load_balancing>
</default>
<!-- Profile that allows only read queries. -->
<readonly>
<readonly>1</readonly>
</readonly>
</profiles>
<!-- Users and ACL. -->
<users>
<!-- If user name was not specified, 'default' user is used. -->
<default>
<!-- Password could be specified in plaintext or in SHA256 (in hex format).
If you want to specify password in plaintext (not recommended), place it in 'password' element.
Example: <password>qwerty</password>.
Password could be empty.
If you want to specify SHA256, place it in 'password_sha256_hex' element.
Example: <password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>
How to generate decent password:
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-'
In first line will be password and in second - corresponding SHA256.
-->
<password></password>
<!-- List of networks with open access.
To open access from everywhere, specify:
<ip>::/0</ip>
To open access only from localhost, specify:
<ip>::1</ip>
<ip>127.0.0.1</ip>
Each element of list has one of the following forms:
<ip> IP-address or network mask. Examples: 213.180.204.3 or 10.0.0.1/8 or 10.0.0.1/255.255.255.0
2a02:6b8::3 or 2a02:6b8::3/64 or 2a02:6b8::3/ffff:ffff:ffff:ffff::.
<host> Hostname. Example: server01.yandex.ru.
To check access, DNS query is performed, and all received addresses compared to peer address.
<host_regexp> Regular expression for host names. Example, ^server\d\d-\d\d-\d\.yandex\.ru$
To check access, DNS PTR query is performed for peer address and then regexp is applied.
Then, for result of PTR query, another DNS query is performed and all received addresses compared to peer address.
Strongly recommended that regexp is ends with $
All results of DNS requests are cached till server restart.
-->
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<!-- Settings profile for user. -->
<profile>default</profile>
<!-- Quota for user. -->
<quota>default</quota>
<!-- For testing the table filters -->
<databases>
<test>
<!-- Simple expression filter -->
<filtered_table1>
<filter>a = 1</filter>
</filtered_table1>
<!-- Complex expression filter -->
<filtered_table2>
<filter>a + b &lt; 1 or c - d &gt; 5</filter>
</filtered_table2>
<!-- Filter with ALIAS column -->
<filtered_table3>
<filter>c = 1</filter>
</filtered_table3>
</test>
</databases>
</default>
<!-- Example of user with readonly access. -->
<!-- <readonly>
<password></password>
<networks incl="networks" replace="replace">
<ip>::1</ip>
<ip>127.0.0.1</ip>
</networks>
<profile>readonly</profile>
<quota>default</quota>
</readonly> -->
</users>
<!-- Quotas. -->
<quotas>
<!-- Name of quota. -->
<default>
<!-- Limits for time interval. You could specify many intervals with different limits. -->
<interval>
<!-- Length of interval. -->
<duration>3600</duration>
<!-- No limits. Just calculate resource usage for time interval. -->
<queries>0</queries>
<errors>0</errors>
<result_rows>0</result_rows>
<read_rows>0</read_rows>
<execution_time>0</execution_time>
</interval>
</default>
</quotas>
</yandex>

View File

@ -0,0 +1,22 @@
import os
import pytest
from helpers.cluster import ClickHouseCluster
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
config_dir = os.path.join(SCRIPT_DIR, './configs')
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', config_dir = config_dir)
caught_exception = ""
@pytest.fixture(scope="module")
def start_cluster():
global caught_exception
try:
cluster.start()
except Exception as e:
caught_exception = str(e)
def test_work(start_cluster):
assert caught_exception.find("Root element doesn't have the corresponding root element as the config file.") != -1

View File

@ -1,5 +1,5 @@
<?xml version="1.0"?> <?xml version="1.0"?>
<dictionaries> <yandex>
<dictionary> <dictionary>
<name>cmd</name> <name>cmd</name>
<source> <source>
@ -16,4 +16,4 @@
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </yandex>

View File

@ -1,30 +1,29 @@
<yandex>
<dictionaries> <dictionary>
<dictionary> <name>dep_x</name>
<name>dep_x</name> <source>
<source> <clickhouse>
<clickhouse> <host>localhost</host>
<host>localhost</host> <port>9000</port>
<port>9000</port> <user>default</user>
<user>default</user> <password></password>
<password></password> <db>dict</db>
<db>dict</db> <table>dep_z</table>
<table>dep_z</table> </clickhouse>
</clickhouse> </source>
</source> <lifetime>5</lifetime>
<lifetime>5</lifetime> <layout>
<layout> <flat/>
<flat/> </layout>
</layout> <structure>
<structure> <id>
<id> <name>id</name>
<name>id</name> </id>
</id> <attribute>
<attribute> <name>a</name>
<name>a</name> <type>String</type>
<type>String</type> <null_value>XX</null_value>
<null_value>XX</null_value> </attribute>
</attribute> </structure>
</structure> </dictionary>
</dictionary> </yandex>
</dictionaries>

View File

@ -1,40 +1,39 @@
<yandex>
<dictionaries>
<dictionary> <dictionary>
<name>dep_y</name> <name>dep_y</name>
<source> <source>
<clickhouse> <clickhouse>
<host>localhost</host> <host>localhost</host>
<port>9000</port> <port>9000</port>
<user>default</user> <user>default</user>
<password></password> <password></password>
<db>test</db> <db>test</db>
<table>small_dict_source</table> <table>small_dict_source</table>
</clickhouse> </clickhouse>
</source> </source>
<lifetime>5</lifetime> <lifetime>5</lifetime>
<layout> <layout>
<flat/> <flat/>
</layout> </layout>
<structure> <structure>
<id> <id>
<name>id</name> <name>id</name>
</id> </id>
<attribute> <attribute>
<name>b</name> <name>b</name>
<type>Int32</type> <type>Int32</type>
<null_value>-1</null_value> <null_value>-1</null_value>
</attribute> </attribute>
<attribute> <attribute>
<name>c</name> <name>c</name>
<type>Float64</type> <type>Float64</type>
<null_value>-2</null_value> <null_value>-2</null_value>
</attribute> </attribute>
<attribute> <attribute>
<name>a</name> <name>a</name>
<type>String</type> <type>String</type>
<null_value>YY</null_value> <null_value>YY</null_value>
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </yandex>

View File

@ -1,4 +1,4 @@
<yandex>
<dictionaries> <dictionaries>
<dictionary> <dictionary>
<name>dep_z</name> <name>dep_z</name>
@ -34,3 +34,4 @@
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </dictionaries>
</yandex>

View File

@ -1,36 +1,53 @@
<?xml version="1.0"?> <?xml version="1.0"?>
<dictionaries> <yandex>
<dictionary> <dictionary>
<name>file</name> <name>file</name>
<source> <source>
<file> <file>
<path>/etc/clickhouse-server/config.d/dictionary_preset_file.txt</path> <path>/etc/clickhouse-server/config.d/dictionary_preset_file.txt</path>
<format>TabSeparated</format> <format>TabSeparated</format>
</file> </file>
</source> </source>
<lifetime>1</lifetime> <lifetime>1</lifetime>
<layout><flat/></layout> <layout><flat/></layout>
<structure><id><name>key</name> </id> <structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type> <attribute><name>a</name><type>Int32</type>
<null_value>0</null_value> <null_value>0</null_value>
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
<dictionary> <dictionary>
<name>no_file</name> <name>no_file</name>
<source> <source>
<file> <file>
<path>/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt</path> <path>/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt</path>
<format>TabSeparated</format> <format>TabSeparated</format>
</file> </file>
</source> </source>
<lifetime>1</lifetime> <lifetime>1</lifetime>
<layout><flat/></layout> <layout><flat/></layout>
<structure><id><name>key</name> </id> <structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type> <attribute><name>a</name><type>Int32</type>
<null_value>0</null_value> <null_value>0</null_value>
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries>
<dictionary>
<name>no_file_2</name>
<source>
<file>
<path>/etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt</path>
<format>TabSeparated</format>
</file>
</source>
<lifetime>1</lifetime>
<layout><flat/></layout>
<structure><id><name>key</name> </id>
<attribute><name>a</name><type>Int32</type>
<null_value>0</null_value>
</attribute>
</structure>
</dictionary>
</yandex>

View File

@ -1,5 +1,5 @@
<?xml version="1.0"?> <?xml version="1.0"?>
<dictionaries> <yandex>
<dictionary> <dictionary>
<name>longload</name> <name>longload</name>
<source> <source>
@ -16,4 +16,4 @@
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </yandex>

View File

@ -52,32 +52,32 @@ def generate_structure():
def generate_dictionaries(path, structure): def generate_dictionaries(path, structure):
dictionary_skeleton = ''' dictionary_skeleton = '''
<dictionaries> <yandex>
<dictionary> <dictionary>
<name>{name}</name> <name>{name}</name>
<source> <source>
{source} {source}
</source> </source>
<lifetime> <lifetime>
<min>0</min> <min>0</min>
<max>0</max> <max>0</max>
</lifetime> </lifetime>
<layout> <layout>
{layout} {layout}
</layout> </layout>
<structure> <structure>
{key} {key}
%s %s
{parent} {parent}
</structure> </structure>
</dictionary> </dictionary>
</dictionaries>''' </yandex>'''
attribute_skeleton = ''' attribute_skeleton = '''
<attribute> <attribute>
<name>%s_</name> <name>%s_</name>

View File

@ -294,7 +294,7 @@ def test_reload_after_loading(started_cluster):
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "83\n" assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "83\n"
def test_reload_after_fail(started_cluster): def test_reload_after_fail_by_system_reload(started_cluster):
query = instance.query query = instance.query
# dictionaries_lazy_load == false, so this dictionary is not loaded. # dictionaries_lazy_load == false, so this dictionary is not loaded.
@ -321,3 +321,32 @@ def test_reload_after_fail(started_cluster):
query("SYSTEM RELOAD DICTIONARY 'no_file'") query("SYSTEM RELOAD DICTIONARY 'no_file'")
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n" query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file") == "LOADED" assert get_status("no_file") == "LOADED"
def test_reload_after_fail_by_timer(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so this dictionary is not loaded.
assert get_status("no_file_2") == "NOT_LOADED"
# We expect an error because the file source doesn't exist.
expected_error = "No such file"
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
assert get_status("no_file_2") == "FAILED"
# Passed time should not change anything now, the status is still FAILED.
time.sleep(6);
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
assert get_status("no_file_2") == "FAILED"
# Creating the file source makes the dictionary able to load.
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt")
time.sleep(6);
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file_2") == "LOADED"
# Removing the file source should not spoil the loaded dictionary.
instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt")
time.sleep(6);
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file_2") == "LOADED"

View File

@ -290,19 +290,19 @@ class Dictionary(object):
def generate_config(self): def generate_config(self):
with open(self.config_path, 'w') as result: with open(self.config_path, 'w') as result:
result.write(''' result.write('''
<dictionaries> <yandex>
<dictionary> <dictionary>
<lifetime> <lifetime>
<min>3</min> <min>3</min>
<max>5</max> <max>5</max>
</lifetime> </lifetime>
<name>{name}</name> <name>{name}</name>
{structure} {structure}
<source> <source>
{source} {source}
</source> </source>
</dictionary> </dictionary>
</dictionaries> </yandex>
'''.format( '''.format(
name=self.name, name=self.name,
structure=self.structure.get_structure_str(), structure=self.structure.get_structure_str(),

View File

@ -3,5 +3,6 @@ FROM php:7.3-cli
COPY ./client.crt client.crt COPY ./client.crt client.crt
COPY ./client.key client.key COPY ./client.key client.key
COPY ./test.php test.php COPY ./test.php test.php
COPY ./test_ssl.php test_ssl.php
RUN docker-php-ext-install pdo pdo_mysql RUN docker-php-ext-install pdo pdo_mysql

View File

@ -12,9 +12,6 @@ $options = [
PDO::ATTR_DEFAULT_FETCH_MODE => PDO::FETCH_ASSOC, PDO::ATTR_DEFAULT_FETCH_MODE => PDO::FETCH_ASSOC,
PDO::ATTR_EMULATE_PREPARES => false, PDO::ATTR_EMULATE_PREPARES => false,
PDO::MYSQL_ATTR_DIRECT_QUERY => true, PDO::MYSQL_ATTR_DIRECT_QUERY => true,
PDO::MYSQL_ATTR_SSL_CERT => "client.crt",
PDO::MYSQL_ATTR_SSL_KEY => "client.key",
PDO::MYSQL_ATTR_SSL_VERIFY_SERVER_CERT => 0,
]; ];
$pdo = new PDO($dsn, $user, $pass, $options); $pdo = new PDO($dsn, $user, $pass, $options);

View File

@ -0,0 +1,27 @@
<?php
$host = $argv[1];
$db = "system";
$user = $argv[3];
$pass = $argv[4];
$charset = "utf8mb4";
$port = $argv[2];
$dsn = "mysql:host=$host;port=$port;dbname=$db;charset=$charset";
$options = [
PDO::ATTR_ERRMODE => PDO::ERRMODE_EXCEPTION,
PDO::ATTR_DEFAULT_FETCH_MODE => PDO::FETCH_ASSOC,
PDO::ATTR_EMULATE_PREPARES => false,
PDO::MYSQL_ATTR_DIRECT_QUERY => true,
PDO::MYSQL_ATTR_SSL_CERT => "client.crt",
PDO::MYSQL_ATTR_SSL_KEY => "client.key",
PDO::MYSQL_ATTR_SSL_VERIFY_SERVER_CERT => 0,
];
$pdo = new PDO($dsn, $user, $pass, $options);
$stmt = $pdo->query("SELECT name FROM tables WHERE name = 'tables'");
foreach ($stmt as $row)
{
echo $row["name"] . "\n";
}
?>

View File

@ -152,3 +152,7 @@ def test_php_client(server_address, php_container):
code, (stdout, stderr) = php_container.exec_run('php -f test.php {host} {port} default 123 '.format(host=server_address, port=server_port), demux=True) code, (stdout, stderr) = php_container.exec_run('php -f test.php {host} {port} default 123 '.format(host=server_address, port=server_port), demux=True)
assert code == 0 assert code == 0
assert stdout == 'tables\n' assert stdout == 'tables\n'
code, (stdout, stderr) = php_container.exec_run('php -f test_ssl.php {host} {port} default 123 '.format(host=server_address, port=server_port), demux=True)
assert code == 0
assert stdout == 'tables\n'

View File

@ -1,38 +1,38 @@
<dictionaries> <yandex>
<dictionary> <dictionary>
<name>postgres_odbc_hashed</name> <name>postgres_odbc_hashed</name>
<source> <source>
<odbc> <odbc>
<table>clickhouse.test_table</table> <table>clickhouse.test_table</table>
<connection_string>DSN=postgresql_odbc;</connection_string> <connection_string>DSN=postgresql_odbc;</connection_string>
<db>postgres</db> <db>postgres</db>
</odbc> </odbc>
</source> </source>
<lifetime> <lifetime>
<min>5</min> <min>5</min>
<max>5</max> <max>5</max>
</lifetime> </lifetime>
<layout> <layout>
<hashed /> <hashed />
</layout> </layout>
<structure> <structure>
<id> <id>
<name>column1</name> <name>column1</name>
</id> </id>
<attribute> <attribute>
<name>column1</name> <name>column1</name>
<type>Int64</type> <type>Int64</type>
<null_value>1</null_value> <null_value>1</null_value>
</attribute> </attribute>
<attribute> <attribute>
<name>column2</name> <name>column2</name>
<type>String</type> <type>String</type>
<null_value>''</null_value> <null_value>''</null_value>
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </yandex>

View File

@ -1,46 +1,46 @@
<dictionaries> <yandex>
<dictionary> <dictionary>
<name>sqlite3_odbc_cached</name> <name>sqlite3_odbc_cached</name>
<source> <source>
<odbc> <odbc>
<table>t3</table> <table>t3</table>
<connection_string>DSN=sqlite3_odbc</connection_string> <connection_string>DSN=sqlite3_odbc</connection_string>
</odbc> </odbc>
</source> </source>
<layout> <layout>
<cache><size_in_cells>128</size_in_cells></cache> <cache><size_in_cells>128</size_in_cells></cache>
</layout> </layout>
<lifetime> <lifetime>
<min>1</min> <min>1</min>
<max>1</max> <max>1</max>
</lifetime> </lifetime>
<structure> <structure>
<id> <id>
<name>X</name> <name>X</name>
</id> </id>
<attribute> <attribute>
<name>X</name> <name>X</name>
<type>Int64</type> <type>Int64</type>
<null_value>1</null_value> <null_value>1</null_value>
</attribute> </attribute>
<attribute> <attribute>
<name>Y</name> <name>Y</name>
<type>Int64</type> <type>Int64</type>
<null_value>1</null_value> <null_value>1</null_value>
</attribute> </attribute>
<attribute> <attribute>
<name>Z</name> <name>Z</name>
<type>UInt8</type> <type>UInt8</type>
<null_value>1</null_value> <null_value>1</null_value>
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </yandex>

View File

@ -1,46 +1,46 @@
<dictionaries> <yandex>
<dictionary> <dictionary>
<name>sqlite3_odbc_hashed</name> <name>sqlite3_odbc_hashed</name>
<source> <source>
<odbc> <odbc>
<table>t2</table> <table>t2</table>
<connection_string>DSN=sqlite3_odbc</connection_string> <connection_string>DSN=sqlite3_odbc</connection_string>
<invalidate_query>SELECT Z from t2 where X=1</invalidate_query> <invalidate_query>SELECT Z from t2 where X=1</invalidate_query>
</odbc> </odbc>
</source> </source>
<lifetime> <lifetime>
<min>1</min> <min>1</min>
<max>1</max> <max>1</max>
</lifetime> </lifetime>
<layout> <layout>
<hashed /> <hashed />
</layout> </layout>
<structure> <structure>
<id> <id>
<name>X</name> <name>X</name>
</id> </id>
<attribute> <attribute>
<name>X</name> <name>X</name>
<type>Int64</type> <type>Int64</type>
<null_value>1</null_value> <null_value>1</null_value>
</attribute> </attribute>
<attribute> <attribute>
<name>Y</name> <name>Y</name>
<type>Int64</type> <type>Int64</type>
<null_value>1</null_value> <null_value>1</null_value>
</attribute> </attribute>
<attribute> <attribute>
<name>Z</name> <name>Z</name>
<type>UInt8</type> <type>UInt8</type>
<null_value>1</null_value> <null_value>1</null_value>
</attribute> </attribute>
</structure> </structure>
</dictionary> </dictionary>
</dictionaries> </yandex>

View File

@ -140,6 +140,7 @@ def test_kafka_settings_old_syntax(kafka_cluster):
result += instance.query('SELECT * FROM test.kafka') result += instance.query('SELECT * FROM test.kafka')
if kafka_check_result(result): if kafka_check_result(result):
break break
time.sleep(0.5)
kafka_check_result(result, True) kafka_check_result(result, True)
@ -170,10 +171,11 @@ def test_kafka_settings_new_syntax(kafka_cluster):
kafka_produce('new', messages) kafka_produce('new', messages)
result = '' result = ''
while True: for i in range(50):
result += instance.query('SELECT * FROM test.kafka') result += instance.query('SELECT * FROM test.kafka')
if kafka_check_result(result): if kafka_check_result(result):
break break
time.sleep(0.5)
kafka_check_result(result, True) kafka_check_result(result, True)
@ -194,10 +196,11 @@ def test_kafka_csv_with_delimiter(kafka_cluster):
kafka_produce('csv', messages) kafka_produce('csv', messages)
result = '' result = ''
while True: for i in range(50):
result += instance.query('SELECT * FROM test.kafka') result += instance.query('SELECT * FROM test.kafka')
if kafka_check_result(result): if kafka_check_result(result):
break break
time.sleep(0.5)
kafka_check_result(result, True) kafka_check_result(result, True)
@ -218,10 +221,11 @@ def test_kafka_tsv_with_delimiter(kafka_cluster):
kafka_produce('tsv', messages) kafka_produce('tsv', messages)
result = '' result = ''
while True: for i in range(50):
result += instance.query('SELECT * FROM test.kafka') result += instance.query('SELECT * FROM test.kafka')
if kafka_check_result(result): if kafka_check_result(result):
break break
time.sleep(0.5)
kafka_check_result(result, True) kafka_check_result(result, True)
@ -246,10 +250,11 @@ def test_kafka_json_without_delimiter(kafka_cluster):
kafka_produce('json', [messages]) kafka_produce('json', [messages])
result = '' result = ''
while True: for i in range(50):
result += instance.query('SELECT * FROM test.kafka') result += instance.query('SELECT * FROM test.kafka')
if kafka_check_result(result): if kafka_check_result(result):
break break
time.sleep(0.5)
kafka_check_result(result, True) kafka_check_result(result, True)
@ -269,10 +274,11 @@ def test_kafka_protobuf(kafka_cluster):
kafka_produce_protobuf_messages('pb', 21, 29) kafka_produce_protobuf_messages('pb', 21, 29)
result = '' result = ''
while True: for i in range(50):
result += instance.query('SELECT * FROM test.kafka') result += instance.query('SELECT * FROM test.kafka')
if kafka_check_result(result): if kafka_check_result(result):
break break
time.sleep(0.5)
kafka_check_result(result, True) kafka_check_result(result, True)
@ -299,11 +305,11 @@ def test_kafka_materialized_view(kafka_cluster):
messages.append(json.dumps({'key': i, 'value': i})) messages.append(json.dumps({'key': i, 'value': i}))
kafka_produce('mv', messages) kafka_produce('mv', messages)
while True: for i in range(50):
time.sleep(1)
result = instance.query('SELECT * FROM test.view') result = instance.query('SELECT * FROM test.view')
if kafka_check_result(result): if kafka_check_result(result):
break break
time.sleep(0.5)
kafka_check_result(result, True) kafka_check_result(result, True)
instance.query(''' instance.query('''
@ -348,11 +354,11 @@ def test_kafka_flush_on_big_message(kafka_cluster):
except kafka.errors.GroupCoordinatorNotAvailableError: except kafka.errors.GroupCoordinatorNotAvailableError:
continue continue
while True: for i in range(50):
time.sleep(1)
result = instance.query('SELECT count() FROM test.view') result = instance.query('SELECT count() FROM test.view')
if int(result) == kafka_messages*batch_messages: if int(result) == kafka_messages*batch_messages:
break break
time.sleep(0.5)
assert int(result) == kafka_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result) assert int(result) == kafka_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result)
@ -378,11 +384,11 @@ def test_kafka_virtual_columns(kafka_cluster):
kafka_produce('virt1', [messages]) kafka_produce('virt1', [messages])
result = '' result = ''
while True: for i in range(50):
time.sleep(1)
result += instance.query('SELECT _key, key, _topic, value, _offset FROM test.kafka') result += instance.query('SELECT _key, key, _topic, value, _offset FROM test.kafka')
if kafka_check_result(result, False, 'test_kafka_virtual1.reference'): if kafka_check_result(result, False, 'test_kafka_virtual1.reference'):
break break
time.sleep(0.5)
kafka_check_result(result, True, 'test_kafka_virtual1.reference') kafka_check_result(result, True, 'test_kafka_virtual1.reference')
@ -409,11 +415,11 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster):
messages.append(json.dumps({'key': i, 'value': i})) messages.append(json.dumps({'key': i, 'value': i}))
kafka_produce('virt2', messages) kafka_produce('virt2', messages)
while True: for i in range(50):
time.sleep(1)
result = instance.query('SELECT kafka_key, key, topic, value, offset FROM test.view') result = instance.query('SELECT kafka_key, key, topic, value, offset FROM test.view')
if kafka_check_result(result, False, 'test_kafka_virtual2.reference'): if kafka_check_result(result, False, 'test_kafka_virtual2.reference'):
break break
time.sleep(0.5)
kafka_check_result(result, True, 'test_kafka_virtual2.reference') kafka_check_result(result, True, 'test_kafka_virtual2.reference')
instance.query(''' instance.query('''

View File

@ -8,15 +8,16 @@ select * from (select toLowCardinality(toNullable(dummy)) as val from system.one
select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val;
select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val;
select '-'; select '-';
select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; select * from (select dummy as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; -- { serverError 352 }
select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; select * from (select dummy as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1;
select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) on val + 0 = val * 1; select * from (select toLowCardinality(dummy) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1;
select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) on val + 0 = val * 1; select * from (select dummy as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1;
select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select dummy as val from system.one) on val + 0 = val * 1; select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1;
select * from (select dummy as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) on val + 0 = val * 1; select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select dummy as rval from system.one) on val + 0 = rval * 1;
select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(dummy) as val from system.one) on val + 0 = val * 1; select * from (select dummy as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1;
select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) on val + 0 = val * 1; select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(dummy) as rval from system.one) on val + 0 = rval * 1;
select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) on val + 0 = val * 1; select * from (select toLowCardinality(dummy) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1;
select * from (select toLowCardinality(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as rval from system.one) on val + 0 = rval * 1;
select '-'; select '-';
select * from (select number as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1; select * from (select number as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1;
select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1; select * from (select toLowCardinality(number) as l from system.numbers limit 3) any left join (select number as r from system.numbers limit 3) on l + 1 = r * 1;

View File

@ -16,7 +16,6 @@ SELECT * FROM (SELECT id FROM test_00808 GROUP BY id LIMIT 1 BY id) WHERE id = 1
SET force_primary_key = 1; SET force_primary_key = 1;
SELECT '-------FORCE PRIMARY KEY-------'; SELECT '-------FORCE PRIMARY KEY-------';
SELECT * FROM (SELECT * FROM test_00808 FINAL) WHERE id = 1; -- { serverError 277 }
SELECT * FROM (SELECT * FROM test_00808 LIMIT 1) WHERE id = 1; -- { serverError 277 } SELECT * FROM (SELECT * FROM test_00808 LIMIT 1) WHERE id = 1; -- { serverError 277 }
SELECT * FROM (SELECT id FROM test_00808 GROUP BY id LIMIT 1 BY id) WHERE id = 1; -- { serverError 277 } SELECT * FROM (SELECT id FROM test_00808 GROUP BY id LIMIT 1 BY id) WHERE id = 1; -- { serverError 277 }

View File

@ -143,3 +143,9 @@ message StrPerson {
MeasureUnits measureUnits = 21; MeasureUnits measureUnits = 21;
NestinessA nestiness_a = 22; NestinessA nestiness_a = 22;
}; };
message NumberAndSquare
{
uint32 number = 1;
uint64 square = 2;
};

View File

@ -1,4 +1,5 @@
echo -ne '\xf3\x01\x0a\x24\x61\x37\x35\x32\x32\x31\x35\x38\x2d\x33\x64\x34\x31\x2d\x34\x62\x37\x37\x2d\x61\x64\x36\x39\x2d\x36\x63\x35\x39\x38\x65\x65\x35\x35\x63\x34\x39\x12\x04\x49\x76\x61\x6e\x1a\x06\x50\x65\x74\x72\x6f\x76\x20\x01\x28\xaf\x1f\x32\x03\x70\x6e\x67\x3a\x0c\x2b\x37\x34\x39\x35\x31\x32\x33\x34\x35\x36\x37\x40\x01\x4d\xfc\xd0\x30\x5c\x50\x26\x58\x09\x62\x09\x59\x65\x73\x74\x65\x72\x64\x61\x79\x62\x07\x46\x6c\x6f\x77\x65\x72\x73\x6a\x04\xff\x01\x00\x00\x72\x06\x4d\x6f\x73\x63\x6f\x77\x7a\x08\x4b\x03\x5f\x42\x72\x7d\x16\x42\x81\x01\x1f\x85\xeb\x51\xb8\x1e\x09\x40\x89\x01\x33\x33\x33\x33\x33\xc3\x6a\x40\x95\x01\xcd\xcc\xcc\x3d\x9d\x01\x9a\x99\xb9\x40\xa0\x01\x80\xc4\xd7\x8d\x7f\xaa\x01\x0c\x0a\x05\x6d\x65\x74\x65\x72\x15\x00\x00\x80\x3f\xaa\x01\x11\x0a\x0a\x63\x65\x6e\x74\x69\x6d\x65\x74\x65\x72\x15\x0a\xd7\x23\x3c\xaa\x01\x10\x0a\x09\x6b\x69\x6c\x6f\x6d\x65\x74\x65\x72\x15\x00\x00\x7a\x44\xb2\x01\x10\x0a\x0e\xa2\x06\x0b\x0a\x09\x08\xf4\x03\x12\x04\xf5\x03\xf6\x03\x7e\x0a\x24\x63\x36\x39\x34\x61\x64\x38\x61\x2d\x66\x37\x31\x34\x2d\x34\x65\x61\x33\x2d\x39\x30\x37\x64\x2d\x66\x64\x35\x34\x66\x62\x32\x35\x64\x39\x62\x35\x12\x07\x4e\x61\x74\x61\x6c\x69\x61\x1a\x08\x53\x6f\x6b\x6f\x6c\x6f\x76\x61\x28\xa6\x3f\x32\x03\x6a\x70\x67\x50\x1a\x58\x0b\x6a\x04\x64\xc8\x01\x32\x72\x08\x50\x6c\x79\x6d\x6f\x75\x74\x68\x7a\x08\x6a\x9d\x49\x42\x46\x8c\x84\xc0\x81\x01\x6e\x86\x1b\xf0\xf9\x21\x09\x40\x95\x01\x42\x60\xe5\x3b\x9d\x01\xcd\xcc\xac\x40\xa0\x01\xff\xff\xa9\xce\x93\x8c\x09\xc0\x01\x0a\x24\x61\x37\x64\x61\x31\x61\x61\x36\x2d\x66\x34\x32\x35\x2d\x34\x37\x38\x39\x2d\x38\x39\x34\x37\x2d\x62\x30\x33\x34\x37\x38\x36\x65\x64\x33\x37\x34\x12\x06\x56\x61\x73\x69\x6c\x79\x1a\x07\x53\x69\x64\x6f\x72\x6f\x76\x20\x01\x28\xfb\x48\x32\x03\x62\x6d\x70\x3a\x0d\x2b\x34\x34\x32\x30\x31\x32\x33\x34\x35\x36\x37\x38\x40\x01\x4d\x50\xe0\x27\x5c\x50\x17\x58\x04\x62\x05\x53\x75\x6e\x6e\x79\x6a\x05\xfa\x01\xf4\x01\x0a\x72\x08\x4d\x75\x72\x6d\x61\x6e\x73\x6b\x7a\x08\xfd\xf0\x89\x42\xc8\x4c\x04\x42\x81\x01\x11\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x37\x42\x95\x01\x00\x00\x48\x44\x9d\x01\xcd\xcc\x4c\xc0\xa0\x01\x80\xd4\x9f\x93\x01\xaa\x01\x0c\x0a\x05\x70\x6f\x75\x6e\x64\x15\x00\x00\x80\x41\xb2\x01\x0a\x0a\x08\xa2\x06\x05\x0a\x03\x08\xf7\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO table_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'" echo -ne '\xf3\x01\x0a\x24\x61\x37\x35\x32\x32\x31\x35\x38\x2d\x33\x64\x34\x31\x2d\x34\x62\x37\x37\x2d\x61\x64\x36\x39\x2d\x36\x63\x35\x39\x38\x65\x65\x35\x35\x63\x34\x39\x12\x04\x49\x76\x61\x6e\x1a\x06\x50\x65\x74\x72\x6f\x76\x20\x01\x28\xaf\x1f\x32\x03\x70\x6e\x67\x3a\x0c\x2b\x37\x34\x39\x35\x31\x32\x33\x34\x35\x36\x37\x40\x01\x4d\xfc\xd0\x30\x5c\x50\x26\x58\x09\x62\x09\x59\x65\x73\x74\x65\x72\x64\x61\x79\x62\x07\x46\x6c\x6f\x77\x65\x72\x73\x6a\x04\xff\x01\x00\x00\x72\x06\x4d\x6f\x73\x63\x6f\x77\x7a\x08\x4b\x03\x5f\x42\x72\x7d\x16\x42\x81\x01\x1f\x85\xeb\x51\xb8\x1e\x09\x40\x89\x01\x33\x33\x33\x33\x33\xc3\x6a\x40\x95\x01\xcd\xcc\xcc\x3d\x9d\x01\x9a\x99\xb9\x40\xa0\x01\x80\xc4\xd7\x8d\x7f\xaa\x01\x0c\x0a\x05\x6d\x65\x74\x65\x72\x15\x00\x00\x80\x3f\xaa\x01\x11\x0a\x0a\x63\x65\x6e\x74\x69\x6d\x65\x74\x65\x72\x15\x0a\xd7\x23\x3c\xaa\x01\x10\x0a\x09\x6b\x69\x6c\x6f\x6d\x65\x74\x65\x72\x15\x00\x00\x7a\x44\xb2\x01\x10\x0a\x0e\xa2\x06\x0b\x0a\x09\x08\xf4\x03\x12\x04\xf5\x03\xf6\x03\x7e\x0a\x24\x63\x36\x39\x34\x61\x64\x38\x61\x2d\x66\x37\x31\x34\x2d\x34\x65\x61\x33\x2d\x39\x30\x37\x64\x2d\x66\x64\x35\x34\x66\x62\x32\x35\x64\x39\x62\x35\x12\x07\x4e\x61\x74\x61\x6c\x69\x61\x1a\x08\x53\x6f\x6b\x6f\x6c\x6f\x76\x61\x28\xa6\x3f\x32\x03\x6a\x70\x67\x50\x1a\x58\x0b\x6a\x04\x64\xc8\x01\x32\x72\x08\x50\x6c\x79\x6d\x6f\x75\x74\x68\x7a\x08\x6a\x9d\x49\x42\x46\x8c\x84\xc0\x81\x01\x6e\x86\x1b\xf0\xf9\x21\x09\x40\x95\x01\x42\x60\xe5\x3b\x9d\x01\xcd\xcc\xac\x40\xa0\x01\xff\xff\xa9\xce\x93\x8c\x09\xc0\x01\x0a\x24\x61\x37\x64\x61\x31\x61\x61\x36\x2d\x66\x34\x32\x35\x2d\x34\x37\x38\x39\x2d\x38\x39\x34\x37\x2d\x62\x30\x33\x34\x37\x38\x36\x65\x64\x33\x37\x34\x12\x06\x56\x61\x73\x69\x6c\x79\x1a\x07\x53\x69\x64\x6f\x72\x6f\x76\x20\x01\x28\xfb\x48\x32\x03\x62\x6d\x70\x3a\x0d\x2b\x34\x34\x32\x30\x31\x32\x33\x34\x35\x36\x37\x38\x40\x01\x4d\x50\xe0\x27\x5c\x50\x17\x58\x04\x62\x05\x53\x75\x6e\x6e\x79\x6a\x05\xfa\x01\xf4\x01\x0a\x72\x08\x4d\x75\x72\x6d\x61\x6e\x73\x6b\x7a\x08\xfd\xf0\x89\x42\xc8\x4c\x04\x42\x81\x01\x11\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x37\x42\x95\x01\x00\x00\x48\x44\x9d\x01\xcd\xcc\x4c\xc0\xa0\x01\x80\xd4\x9f\x93\x01\xaa\x01\x0c\x0a\x05\x70\x6f\x75\x6e\x64\x15\x00\x00\x80\x41\xb2\x01\x0a\x0a\x08\xa2\x06\x05\x0a\x03\x08\xf7\x03' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'"
echo -ne '\xb3\x01\x12\x05\x46\x72\x69\x64\x61\x28\x99\xe1\xf3\xd1\x0b\x52\x08\x45\x72\x6d\x61\x6b\x6f\x76\x61\x72\x0c\x00\x00\xdc\x42\x00\x00\x52\x43\x00\x00\x94\x42\x79\x48\xce\x3d\x51\x00\x00\x00\x00\xc8\x02\x14\xc2\x05\x08\x00\x00\x80\x44\x00\x00\x80\x49\x9a\x06\x02\x4b\x42\x9a\x06\x02\x4d\x42\xa1\x06\x00\x00\x00\x00\x00\x00\xe0\x3f\xa8\x06\x2a\xa8\x06\xa8\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\x25\x06\x49\x40\xfa\x06\x02\x34\x30\x90\x08\xe2\x08\xe1\x08\x89\xe6\x6e\xdd\x01\x00\x00\x00\xb0\x09\xc3\x19\xd0\x0c\xb7\x02\xe2\x12\x24\x32\x30\x66\x63\x64\x39\x35\x61\x2d\x33\x33\x32\x64\x2d\x34\x31\x64\x62\x2d\x61\x39\x65\x63\x2d\x31\x36\x31\x66\x36\x34\x34\x64\x30\x35\x39\x63\xa0\x38\xbc\x05\xaa\x38\x02\xbd\x05\xb4\x01\x08\x01\x12\x06\x49\x73\x6f\x6c\x64\x65\x52\x07\x4c\x61\x76\x72\x6f\x76\x61\x72\x0c\x00\x00\x7f\x43\x00\x00\x00\x00\x00\x00\x7f\x43\xaa\x01\x03\x61\x62\x63\xc8\x02\x32\xc2\x05\x08\x00\x00\x00\x41\x00\x00\x80\x3f\x9a\x06\x04\x42\x79\x74\x65\x9a\x06\x03\x42\x69\x74\xa1\x06\x00\x00\x00\x00\x00\x00\x12\x40\xa8\x06\x1a\xa8\x06\xb0\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\xf9\x0f\x49\x40\xc2\x06\x01\x2c\xfa\x06\x02\x33\x32\x90\x08\x78\xe1\x08\x39\x4e\x2b\xfe\xe4\xf5\xff\xff\xb0\x09\xe8\x30\xd8\x12\x01\xe2\x12\x24\x37\x63\x66\x61\x36\x38\x35\x36\x2d\x61\x35\x34\x61\x2d\x34\x37\x38\x36\x2d\x62\x38\x65\x35\x2d\x37\x34\x35\x31\x35\x39\x64\x35\x32\x32\x37\x38\xa0\x38\xbe\x05\xc2\x3e\x05\x15\x00\x00\xb6\x42' | $CLICKHOUSE_CLIENT --query="INSERT INTO table_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'" echo -ne '\xb3\x01\x12\x05\x46\x72\x69\x64\x61\x28\x99\xe1\xf3\xd1\x0b\x52\x08\x45\x72\x6d\x61\x6b\x6f\x76\x61\x72\x0c\x00\x00\xdc\x42\x00\x00\x52\x43\x00\x00\x94\x42\x79\x48\xce\x3d\x51\x00\x00\x00\x00\xc8\x02\x14\xc2\x05\x08\x00\x00\x80\x44\x00\x00\x80\x49\x9a\x06\x02\x4b\x42\x9a\x06\x02\x4d\x42\xa1\x06\x00\x00\x00\x00\x00\x00\xe0\x3f\xa8\x06\x2a\xa8\x06\xa8\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\x25\x06\x49\x40\xfa\x06\x02\x34\x30\x90\x08\xe2\x08\xe1\x08\x89\xe6\x6e\xdd\x01\x00\x00\x00\xb0\x09\xc3\x19\xd0\x0c\xb7\x02\xe2\x12\x24\x32\x30\x66\x63\x64\x39\x35\x61\x2d\x33\x33\x32\x64\x2d\x34\x31\x64\x62\x2d\x61\x39\x65\x63\x2d\x31\x36\x31\x66\x36\x34\x34\x64\x30\x35\x39\x63\xa0\x38\xbc\x05\xaa\x38\x02\xbd\x05\xb4\x01\x08\x01\x12\x06\x49\x73\x6f\x6c\x64\x65\x52\x07\x4c\x61\x76\x72\x6f\x76\x61\x72\x0c\x00\x00\x7f\x43\x00\x00\x00\x00\x00\x00\x7f\x43\xaa\x01\x03\x61\x62\x63\xc8\x02\x32\xc2\x05\x08\x00\x00\x00\x41\x00\x00\x80\x3f\x9a\x06\x04\x42\x79\x74\x65\x9a\x06\x03\x42\x69\x74\xa1\x06\x00\x00\x00\x00\x00\x00\x12\x40\xa8\x06\x1a\xa8\x06\xb0\xff\xff\xff\xff\xff\xff\xff\xff\x01\xb0\x06\x01\xbd\x06\xf9\x0f\x49\x40\xc2\x06\x01\x2c\xfa\x06\x02\x33\x32\x90\x08\x78\xe1\x08\x39\x4e\x2b\xfe\xe4\xf5\xff\xff\xb0\x09\xe8\x30\xd8\x12\x01\xe2\x12\x24\x37\x63\x66\x61\x36\x38\x35\x36\x2d\x61\x35\x34\x61\x2d\x34\x37\x38\x36\x2d\x62\x38\x65\x35\x2d\x37\x34\x35\x31\x35\x39\x64\x35\x32\x32\x37\x38\xa0\x38\xbe\x05\xc2\x3e\x05\x15\x00\x00\xb6\x42' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'"
echo -ne '\xa5\x02\x0a\x24\x61\x61\x30\x65\x35\x61\x30\x36\x2d\x63\x61\x62\x32\x2d\x34\x30\x33\x34\x2d\x61\x36\x61\x32\x2d\x34\x38\x65\x38\x32\x62\x39\x31\x36\x36\x34\x65\x12\x06\x4c\x65\x6f\x6e\x69\x64\x1a\x08\x4b\x69\x72\x69\x6c\x6c\x6f\x76\x22\x04\x6d\x61\x6c\x65\x2a\x0a\x31\x39\x38\x33\x2d\x30\x36\x2d\x32\x34\x3a\x0c\x2b\x37\x34\x39\x35\x30\x32\x37\x35\x38\x36\x34\x42\x01\x31\x4a\x13\x32\x30\x31\x39\x2d\x30\x32\x2d\x30\x34\x20\x30\x39\x3a\x34\x35\x3a\x30\x30\x52\x02\x33\x35\x5a\x06\x63\x61\x6e\x63\x65\x72\x62\x07\x37\x20\x72\x69\x6e\x67\x73\x62\x08\x45\x61\x73\x74\x73\x69\x64\x65\x62\x0b\x4c\x61\x73\x74\x20\x48\x75\x72\x72\x61\x68\x6a\x01\x30\x6a\x01\x30\x6a\x03\x32\x35\x35\x72\x09\x53\x61\x6e\x20\x44\x69\x65\x67\x6f\x7a\x09\x33\x32\x2e\x38\x32\x33\x39\x34\x33\x7a\x0b\x2d\x31\x31\x37\x2e\x30\x38\x31\x33\x32\x37\x82\x01\x09\x33\x2e\x31\x34\x31\x35\x39\x32\x37\x8a\x01\x08\x31\x35\x30\x30\x30\x30\x30\x30\x92\x01\x06\x31\x38\x36\x2e\x37\x35\x9a\x01\x04\x2d\x32\x2e\x31\xa2\x01\x0b\x32\x30\x36\x35\x39\x38\x32\x39\x33\x33\x31\xaa\x01\x18\x0a\x06\x6d\x69\x6e\x75\x74\x65\x0a\x04\x68\x6f\x75\x72\x12\x02\x36\x30\x12\x04\x33\x36\x30\x30\xb2\x01\x08\x0a\x06\x12\x04\x31\x38\x30\x30' | $CLICKHOUSE_CLIENT --query="INSERT INTO table_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'" echo -ne '\xa5\x02\x0a\x24\x61\x61\x30\x65\x35\x61\x30\x36\x2d\x63\x61\x62\x32\x2d\x34\x30\x33\x34\x2d\x61\x36\x61\x32\x2d\x34\x38\x65\x38\x32\x62\x39\x31\x36\x36\x34\x65\x12\x06\x4c\x65\x6f\x6e\x69\x64\x1a\x08\x4b\x69\x72\x69\x6c\x6c\x6f\x76\x22\x04\x6d\x61\x6c\x65\x2a\x0a\x31\x39\x38\x33\x2d\x30\x36\x2d\x32\x34\x3a\x0c\x2b\x37\x34\x39\x35\x30\x32\x37\x35\x38\x36\x34\x42\x01\x31\x4a\x13\x32\x30\x31\x39\x2d\x30\x32\x2d\x30\x34\x20\x30\x39\x3a\x34\x35\x3a\x30\x30\x52\x02\x33\x35\x5a\x06\x63\x61\x6e\x63\x65\x72\x62\x07\x37\x20\x72\x69\x6e\x67\x73\x62\x08\x45\x61\x73\x74\x73\x69\x64\x65\x62\x0b\x4c\x61\x73\x74\x20\x48\x75\x72\x72\x61\x68\x6a\x01\x30\x6a\x01\x30\x6a\x03\x32\x35\x35\x72\x09\x53\x61\x6e\x20\x44\x69\x65\x67\x6f\x7a\x09\x33\x32\x2e\x38\x32\x33\x39\x34\x33\x7a\x0b\x2d\x31\x31\x37\x2e\x30\x38\x31\x33\x32\x37\x82\x01\x09\x33\x2e\x31\x34\x31\x35\x39\x32\x37\x8a\x01\x08\x31\x35\x30\x30\x30\x30\x30\x30\x92\x01\x06\x31\x38\x36\x2e\x37\x35\x9a\x01\x04\x2d\x32\x2e\x31\xa2\x01\x0b\x32\x30\x36\x35\x39\x38\x32\x39\x33\x33\x31\xaa\x01\x18\x0a\x06\x6d\x69\x6e\x75\x74\x65\x0a\x04\x68\x6f\x75\x72\x12\x02\x36\x30\x12\x04\x33\x36\x30\x30\xb2\x01\x08\x0a\x06\x12\x04\x31\x38\x30\x30' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'"
echo -ne '\xdd\x01\x0a\x24\x33\x66\x61\x65\x65\x30\x36\x34\x2d\x63\x34\x66\x37\x2d\x34\x64\x33\x34\x2d\x62\x36\x66\x33\x2d\x38\x64\x38\x31\x63\x32\x62\x36\x61\x31\x35\x64\x12\x04\x4e\x69\x63\x6b\x1a\x0a\x4b\x6f\x6c\x65\x73\x6e\x69\x6b\x6f\x76\x20\x01\x28\xda\x52\x32\x03\x62\x6d\x70\x3a\x0c\x34\x31\x32\x2d\x36\x38\x37\x2d\x35\x30\x30\x37\x40\x01\x4d\x2f\x27\xf2\x5b\x50\x14\x58\x09\x62\x06\x48\x61\x76\x61\x6e\x61\x68\x80\x01\x68\x00\x68\x80\x01\x72\x0a\x50\x69\x74\x74\x73\x62\x75\x72\x67\x68\x7a\x08\x9b\x11\x22\x42\x1f\xe6\x9f\xc2\x81\x01\x28\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x27\x42\x95\x01\x00\x00\x43\x44\x9d\x01\x66\x66\x92\x41\xa0\x01\xce\xdf\xb8\xba\x01\xab\x01\x0d\xcd\xcc\xe2\x41\x0d\xcd\xcc\x4c\x3e\x0d\x00\x00\x80\x3f\x12\x05\x6f\x75\x6e\x63\x65\x12\x05\x63\x61\x72\x61\x74\x12\x04\x67\x72\x61\x6d\xac\x01\xb3\x01\x0b\xa2\x06\x05\x0b\x08\x96\x4a\x0c\x0c\xb4\x01' | $CLICKHOUSE_CLIENT --query="INSERT INTO table_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'" echo -ne '\xdd\x01\x0a\x24\x33\x66\x61\x65\x65\x30\x36\x34\x2d\x63\x34\x66\x37\x2d\x34\x64\x33\x34\x2d\x62\x36\x66\x33\x2d\x38\x64\x38\x31\x63\x32\x62\x36\x61\x31\x35\x64\x12\x04\x4e\x69\x63\x6b\x1a\x0a\x4b\x6f\x6c\x65\x73\x6e\x69\x6b\x6f\x76\x20\x01\x28\xda\x52\x32\x03\x62\x6d\x70\x3a\x0c\x34\x31\x32\x2d\x36\x38\x37\x2d\x35\x30\x30\x37\x40\x01\x4d\x2f\x27\xf2\x5b\x50\x14\x58\x09\x62\x06\x48\x61\x76\x61\x6e\x61\x68\x80\x01\x68\x00\x68\x80\x01\x72\x0a\x50\x69\x74\x74\x73\x62\x75\x72\x67\x68\x7a\x08\x9b\x11\x22\x42\x1f\xe6\x9f\xc2\x81\x01\x28\x2d\x44\x54\xfb\x21\x09\x40\x89\x01\x00\x00\x00\xe8\x76\x48\x27\x42\x95\x01\x00\x00\x43\x44\x9d\x01\x66\x66\x92\x41\xa0\x01\xce\xdf\xb8\xba\x01\xab\x01\x0d\xcd\xcc\xe2\x41\x0d\xcd\xcc\x4c\x3e\x0d\x00\x00\x80\x3f\x12\x05\x6f\x75\x6e\x63\x65\x12\x05\x63\x61\x72\x61\x74\x12\x04\x67\x72\x61\x6d\xac\x01\xb3\x01\x0b\xa2\x06\x05\x0b\x08\x96\x4a\x0c\x0c\xb4\x01' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_persons_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'"
echo -ne '\x04\x08\x02\x10\x04\x00\x04\x08\x03\x10\x09' | $CLICKHOUSE_CLIENT --query="INSERT INTO in_squares_00825 FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare'"

View File

@ -5,3 +5,6 @@ aa0e5a06-cab2-4034-a6a2-48e82b91664e Leonid Kirillov male 1983-06-24 \N +7495027
a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502] a7522158-3d41-4b77-ad69-6c598ee55c49 Ivan Petrov male 1980-12-29 png +74951234567\0 1 2019-01-05 18:45:00 38 capricorn ['Yesterday','Flowers'] [255,0,0] Moscow [55.753216,37.622504] 3.14 214.10 0.1 5.8 17060000000 ['meter','centimeter','kilometer'] [1,0.01,1000] 500 [501,502]
3faee064-c4f7-4d34-b6f3-8d81c2b6a15d Nick Kolesnikov male 1998-12-26 bmp 412-687-5007\0 1 2018-11-19 05:59:59 20 capricorn ['Havana'] [128,0,128] Pittsburgh [40.517192,-79.949456] 3.1415926535898 50000000000.00 780 18.3 195500007 ['ounce','carat','gram'] [28.35,0.2,1] 9494 [] 3faee064-c4f7-4d34-b6f3-8d81c2b6a15d Nick Kolesnikov male 1998-12-26 bmp 412-687-5007\0 1 2018-11-19 05:59:59 20 capricorn ['Havana'] [128,0,128] Pittsburgh [40.517192,-79.949456] 3.1415926535898 50000000000.00 780 18.3 195500007 ['ounce','carat','gram'] [28.35,0.2,1] 9494 []
7cfa6856-a54a-4786-b8e5-745159d52278 Isolde Lavrova female 1987-02-09 \N \N 1 \N 32 aquarius [] [255,0,255] [26.000000,-80.000000] 3.1415998935699463 \N 4.5 25.0 -11111111111111 ['Byte','Bit'] [8,1] 702 [] 7cfa6856-a54a-4786-b8e5-745159d52278 Isolde Lavrova female 1987-02-09 \N \N 1 \N 32 aquarius [] [255,0,255] [26.000000,-80.000000] 3.1415998935699463 \N 4.5 25.0 -11111111111111 ['Byte','Bit'] [8,1] 702 []
0 0
2 4
3 9

View File

@ -7,34 +7,37 @@ set -e -o pipefail
# Run the client. # Run the client.
$CLICKHOUSE_CLIENT --multiquery <<'EOF' $CLICKHOUSE_CLIENT --multiquery <<'EOF'
DROP TABLE IF EXISTS table_00825; DROP TABLE IF EXISTS in_persons_00825;
DROP TABLE IF EXISTS in_squares_00825;
CREATE TABLE table_00825 (uuid UUID, CREATE TABLE in_persons_00825 (uuid UUID,
name String, name String,
surname String, surname String,
gender Enum8('male'=1, 'female'=0), gender Enum8('male'=1, 'female'=0),
birthDate Date, birthDate Date,
photo Nullable(String), photo Nullable(String),
phoneNumber Nullable(FixedString(13)), phoneNumber Nullable(FixedString(13)),
isOnline UInt8, isOnline UInt8,
visitTime Nullable(DateTime), visitTime Nullable(DateTime),
age UInt8, age UInt8,
zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823, zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823,
'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120, 'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120,
'pisces'=219), 'pisces'=219),
songs Array(String), songs Array(String),
color Array(UInt8), color Array(UInt8),
hometown LowCardinality(String), hometown LowCardinality(String),
location Array(Decimal32(6)), location Array(Decimal32(6)),
pi Nullable(Float64), pi Nullable(Float64),
lotteryWin Nullable(Decimal64(2)), lotteryWin Nullable(Decimal64(2)),
someRatio Float32, someRatio Float32,
temperature Decimal32(1), temperature Decimal32(1),
randomBigNumber Int64, randomBigNumber Int64,
measureUnits Nested (unit String, coef Float32), measureUnits Nested (unit String, coef Float32),
nestiness_a_b_c_d Nullable(UInt32), nestiness_a_b_c_d Nullable(UInt32),
`nestiness_a_B.c_E` Array(UInt32) `nestiness_a_B.c_E` Array(UInt32)
) ENGINE = MergeTree ORDER BY tuple(); ) ENGINE = MergeTree ORDER BY tuple();
CREATE TABLE in_squares_00825 (number UInt32, square UInt32) ENGINE = MergeTree ORDER BY tuple();
EOF EOF
# To generate the file 00825_protobuf_format_input.insh use the following commands: # To generate the file 00825_protobuf_format_input.insh use the following commands:
@ -42,6 +45,8 @@ EOF
# build/utils/test-data-generator/ProtobufDelimitedMessagesSerializer # build/utils/test-data-generator/ProtobufDelimitedMessagesSerializer
source $CURDIR/00825_protobuf_format_input.insh source $CURDIR/00825_protobuf_format_input.insh
$CLICKHOUSE_CLIENT --query "SELECT * FROM table_00825 ORDER BY uuid;" $CLICKHOUSE_CLIENT --query "SELECT * FROM in_persons_00825 ORDER BY uuid;"
$CLICKHOUSE_CLIENT --query "SELECT * FROM in_squares_00825 ORDER BY number;"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS table_00825;" $CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS in_persons_00825;"
$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS in_squares_00825;"

View File

@ -11,46 +11,53 @@ set -e -o pipefail
# Run the client. # Run the client.
$CLICKHOUSE_CLIENT --multiquery <<EOF $CLICKHOUSE_CLIENT --multiquery <<EOF
DROP TABLE IF EXISTS table_00825; DROP TABLE IF EXISTS out_persons_00825;
DROP TABLE IF EXISTS out_squares_00825;
CREATE TABLE table_00825 (uuid UUID, CREATE TABLE out_persons_00825 (uuid UUID,
name String, name String,
surname String, surname String,
gender Enum8('male'=1, 'female'=0), gender Enum8('male'=1, 'female'=0),
birthDate Date, birthDate Date,
photo Nullable(String), photo Nullable(String),
phoneNumber Nullable(FixedString(13)), phoneNumber Nullable(FixedString(13)),
isOnline UInt8, isOnline UInt8,
visitTime Nullable(DateTime), visitTime Nullable(DateTime),
age UInt8, age UInt8,
zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823, zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823,
'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120, 'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120,
'pisces'=219), 'pisces'=219),
songs Array(String), songs Array(String),
color Array(UInt8), color Array(UInt8),
hometown LowCardinality(String), hometown LowCardinality(String),
location Array(Decimal32(6)), location Array(Decimal32(6)),
pi Nullable(Float64), pi Nullable(Float64),
lotteryWin Nullable(Decimal64(2)), lotteryWin Nullable(Decimal64(2)),
someRatio Float32, someRatio Float32,
temperature Decimal32(1), temperature Decimal32(1),
randomBigNumber Int64, randomBigNumber Int64,
measureUnits Nested(unit String, coef Float32), measureUnits Nested(unit String, coef Float32),
nestiness_a_b_c_d Nullable(UInt32), nestiness_a_b_c_d Nullable(UInt32),
\`nestiness_a_B.c_E\` Array(UInt32) \`nestiness_a_B.c_E\` Array(UInt32)
) ENGINE = MergeTree ORDER BY tuple(); ) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO table_00825 VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000, ['meter', 'centimeter', 'kilometer'], [1, 0.01, 1000], 500, [501, 502]); CREATE TABLE out_squares_00825 (number UInt32, square UInt64) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO table_00825 VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Plymouth', [50.403724, -4.142123], 3.14159, NULL, 0.007, 5.4, -20000000000000, [], [], NULL, []);
INSERT INTO table_00825 VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000, ['pound'], [16], 503, []);
SELECT * FROM table_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person'; INSERT INTO out_persons_00825 VALUES (toUUID('a7522158-3d41-4b77-ad69-6c598ee55c49'), 'Ivan', 'Petrov', 'male', toDate('1980-12-29'), 'png', '+74951234567', 1, toDateTime('2019-01-05 18:45:00'), 38, 'capricorn', ['Yesterday', 'Flowers'], [255, 0, 0], 'Moscow', [55.753215, 37.622504], 3.14, 214.10, 0.1, 5.8, 17060000000, ['meter', 'centimeter', 'kilometer'], [1, 0.01, 1000], 500, [501, 502]);
INSERT INTO out_persons_00825 VALUES (toUUID('c694ad8a-f714-4ea3-907d-fd54fb25d9b5'), 'Natalia', 'Sokolova', 'female', toDate('1992-03-08'), 'jpg', NULL, 0, NULL, 26, 'pisces', [], [100, 200, 50], 'Plymouth', [50.403724, -4.142123], 3.14159, NULL, 0.007, 5.4, -20000000000000, [], [], NULL, []);
INSERT INTO out_persons_00825 VALUES (toUUID('a7da1aa6-f425-4789-8947-b034786ed374'), 'Vasily', 'Sidorov', 'male', toDate('1995-07-28'), 'bmp', '+442012345678', 1, toDateTime('2018-12-30 00:00:00'), 23, 'leo', ['Sunny'], [250, 244, 10], 'Murmansk', [68.970682, 33.074981], 3.14159265358979, 100000000000, 800, -3.2, 154400000, ['pound'], [16], 503, []);
INSERT INTO out_squares_00825 VALUES (2, 4), (0, 0), (3, 9);
SELECT * FROM out_persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:Person';
SELECT 'ALTERNATIVE->'; SELECT 'ALTERNATIVE->';
SELECT * FROM table_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson'; SELECT * FROM out_persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:AltPerson';
SELECT 'STRINGS->'; SELECT 'STRINGS->';
SELECT * FROM table_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson'; SELECT * FROM out_persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:StrPerson';
SELECT 'SYNTAX2->'; SELECT 'SYNTAX2->';
SELECT * FROM table_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person'; SELECT * FROM out_persons_00825 ORDER BY name FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format_syntax2:Syntax2Person';
SELECT 'SQUARES->';
SELECT * FROM out_squares_00825 ORDER BY number FORMAT Protobuf SETTINGS format_schema = '$CURDIR/00825_protobuf_format:NumberAndSquare';
DROP TABLE IF EXISTS table_00825; DROP TABLE IF EXISTS out_persons_00825;
DROP TABLE IF EXISTS out_squares_00825;
EOF EOF

View File

@ -1,9 +1,11 @@
#!/usr/bin/env bash #!/usr/bin/env bash
CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh . $CURDIR/../shell_config.sh
set -e set -e
# No log lines without query id # No log lines without query id
$CLICKHOUSE_CLIENT --send_logs_level=trace --query_id=hello --query="SELECT count() FROM numbers(10)" 2>&1 | grep -vF ' {hello} ' | grep -P '<\w+>' ||: $CLICKHOUSE_CLIENT --query_id=hello --query="SELECT count() FROM numbers(10)" 2>&1 | grep -vF ' {hello} ' | grep -P '<\w+>' ||:

View File

@ -0,0 +1,56 @@
SET allow_experimental_data_skipping_indices = 1;
DROP TABLE IF EXISTS indexed_table;
CREATE TABLE indexed_table
(
`tm` DateTime,
`log_message` String,
INDEX log_message log_message TYPE tokenbf_v1(4096, 2, 0) GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY (tm)
SETTINGS index_granularity_bytes = 50;
INSERT INTO indexed_table SELECT toDateTime('2019-05-27 10:00:00') + number % 100, 'h' FROM numbers(1000);
INSERT INTO indexed_table
SELECT
toDateTime('2019-05-27 10:00:00') + number % 100,
concat('hhhhhhhhhhhhhhhhhhhhhhhhh', 'xxxxxxxxxxxxxxxxxxxxxxxxxxxx', 'yyyyyyyyyyyyyyyyyyyyyyyyyy', toString(rand()))
FROM numbers(1000);
OPTIMIZE TABLE indexed_table FINAL;
SELECT COUNT() FROM indexed_table WHERE log_message like '%x%';
DROP TABLE IF EXISTS indexed_table;
DROP TABLE IF EXISTS another_indexed_table;
CREATE TABLE another_indexed_table
(
`tm` DateTime,
`log_message` String,
INDEX log_message log_message TYPE tokenbf_v1(4096, 2, 0) GRANULARITY 1
)
ENGINE = MergeTree
ORDER BY (tm)
SETTINGS index_granularity_bytes = 50,
vertical_merge_algorithm_min_rows_to_activate=0,
vertical_merge_algorithm_min_columns_to_activate=0;
INSERT INTO another_indexed_table SELECT toDateTime('2019-05-27 10:00:00') + number % 100, 'h' FROM numbers(1000);
INSERT INTO another_indexed_table
SELECT
toDateTime('2019-05-27 10:00:00') + number % 100,
concat('hhhhhhhhhhhhhhhhhhhhhhhhh', 'xxxxxxxxxxxxxxxxxxxxxxxxxxxx', 'yyyyyyyyyyyyyyyyyyyyyyyyyy', toString(rand()))
FROM numbers(1000);
OPTIMIZE TABLE another_indexed_table FINAL;
SELECT COUNT() FROM another_indexed_table WHERE log_message like '%x%';
DROP TABLE IF EXISTS another_indexed_table;

View File

@ -0,0 +1,5 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (num UInt64, str String) ENGINE = MergeTree ORDER BY num;
INSERT INTO test (num) VALUES (1), (2), (10), (15), (23);
SELECT count(*) FROM test WHERE bitmapContains(bitmapBuild([1, 5, 7, 9]), toUInt32(num));
DROP TABLE test;

View File

@ -0,0 +1,17 @@
DROP TABLE IF EXISTS test_00974;
CREATE TABLE test_00974
(
date Date,
x Int32,
ver UInt64
)
ENGINE = ReplacingMergeTree(date, x, 1);
INSERT INTO test_00974 VALUES ('2019-07-23', 1, 1), ('2019-07-23', 1, 2);
INSERT INTO test_00974 VALUES ('2019-07-23', 2, 1), ('2019-07-23', 2, 2);
SELECT COUNT() FROM (SELECT * FROM test_00974 FINAL) where x = 1 SETTINGS enable_optimize_predicate_expression_to_final_subquery = 0;
SELECT COUNT() FROM (SELECT * FROM test_00974 FINAL) where x = 1 SETTINGS enable_optimize_predicate_expression_to_final_subquery = 1, max_rows_to_read = 2;
DROP TABLE test_00974;

View File

@ -0,0 +1,40 @@
2 y 2 w
2 y 2 w
2 2
2 2
y w
y w
2 2
2 2
y w
y w
y y
y y
y y
y y
2 2
2 2
2 2
2 2
2 2
2 2
2 y 2 w
2 y 2 w
2 2
2 2
y w
y w
2 2
2 2
y w
y w
y y
y y
y y
y y
2 2
2 2
2 2
2 2
2 2
2 2

View File

@ -0,0 +1,69 @@
use test;
drop table if exists t1;
drop table if exists t2;
create table t1 (a UInt32, b String) engine = Memory;
create table t2 (c UInt32, d String) engine = Memory;
insert into t1 values (1, 'x'), (2, 'y'), (3, 'z');
insert into t2 values (2, 'w'), (4, 'y');
set enable_optimize_predicate_expression = 0;
select * from t1 join t2 on a = c;
select * from t1 join t2 on c = a;
select t1.a, t2.c from t1 join t2 on a = c;
select t1.a, t2.c from t1 join t2 on c = a;
select t1.b, t2.d from t1 join t2 on a = c;
select t1.b, t2.d from t1 join t2 on c = a;
select a, c from t1 join t2 on a = c;
select a, c from t1 join t2 on c = a;
select b, d from t1 join t2 on a = c;
select b, d from t1 join t2 on c = a;
select b as a, d as c from t1 join t2 on a = c;
select b as a, d as c from t1 join t2 on c = a;
select b as c, d as a from t1 join t2 on a = c;
select b as c, d as a from t1 join t2 on c = a;
select t1.a as a, t2.c as c from t1 join t2 on a = c;
select t1.a as a, t2.c as c from t1 join t2 on c = a;
select t1.a as c, t2.c as a from t1 join t2 on a = c;
select t1.a as c, t2.c as a from t1 join t2 on c = a;
select t1.a as c, t2.c as a from t1 join t2 on t1.a = t2.c;
select t1.a as c, t2.c as a from t1 join t2 on t2.c = t1.a;
set enable_optimize_predicate_expression = 1;
select * from t1 join t2 on a = c;
select * from t1 join t2 on c = a;
select t1.a, t2.c from t1 join t2 on a = c;
select t1.a, t2.c from t1 join t2 on c = a;
select t1.b, t2.d from t1 join t2 on a = c;
select t1.b, t2.d from t1 join t2 on c = a;
select a, c from t1 join t2 on a = c;
select a, c from t1 join t2 on c = a;
select b, d from t1 join t2 on a = c;
select b, d from t1 join t2 on c = a;
select b as a, d as c from t1 join t2 on a = c;
select b as a, d as c from t1 join t2 on c = a;
select b as c, d as a from t1 join t2 on a = c;
select b as c, d as a from t1 join t2 on c = a;
select t1.a as a, t2.c as c from t1 join t2 on a = c;
select t1.a as a, t2.c as c from t1 join t2 on c = a;
select t1.a as c, t2.c as a from t1 join t2 on a = c;
select t1.a as c, t2.c as a from t1 join t2 on c = a;
select t1.a as c, t2.c as a from t1 join t2 on t1.a = t2.c;
select t1.a as c, t2.c as a from t1 join t2 on t2.c = t1.a;
drop table t1;
drop table t2;

View File

@ -0,0 +1,5 @@
2015-12-01 0 0
2015-12-02 1 1
2015-12-03 0 2
2015-12-04 0 3
2015-12-05 0 4

View File

@ -0,0 +1,20 @@
SELECT
q0.dt,
q0.cnt,
q0.cnt2
FROM
(
SELECT
toDate(addDays(toDate('2015-12-01'), number)) AS dt,
sum(number) AS cnt
FROM numbers(2)
GROUP BY dt
) AS q0
ALL FULL OUTER JOIN
(
SELECT
toDate(addDays(toDate('2015-12-01'), number)) AS dt,
sum(number) AS cnt2
FROM numbers(5)
GROUP BY dt
) AS q1 ON q0.dt = q1.dt

View File

@ -0,0 +1,4 @@
0 0
1
1000000000 0
1

View File

@ -0,0 +1,14 @@
SET query_profiler_real_time_period_ns = 100000000;
SET log_queries = 1;
SELECT sleep(0.5), ignore('test real time query profiler');
SET log_queries = 0;
SYSTEM FLUSH LOGS;
WITH symbolizeAddress(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test real time query profiler%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%FunctionSleep%';
SET query_profiler_real_time_period_ns = 0;
SET query_profiler_cpu_time_period_ns = 100000000;
SET log_queries = 1;
SELECT count(), ignore('test cpu time query profiler') FROM numbers(1000000000);
SET log_queries = 0;
SYSTEM FLUSH LOGS;
WITH symbolizeAddress(arrayJoin(trace)) AS symbol SELECT count() > 0 FROM system.trace_log t WHERE event_date >= yesterday() AND query_id = (SELECT query_id FROM system.query_log WHERE event_date >= yesterday() AND query LIKE '%test cpu time query profiler%' ORDER BY event_time DESC LIMIT 1) AND symbol LIKE '%Numbers%';

View File

@ -32,7 +32,7 @@ Possible values:
- 0 — Disabled. - 0 — Disabled.
- 1 — Enabled. - 1 — Enabled.
Default value: 0. Default value: 1.
**Usage** **Usage**

View File

@ -42,7 +42,7 @@ A structure can contain either `<id>` or `<key>` .
### Numeric Key ### Numeric Key
Format: `UInt64`. Type: `UInt64`.
Configuration example: Configuration example:
@ -54,7 +54,7 @@ Configuration example:
Configuration fields: Configuration fields:
- name The name of the column with keys. - `name` The name of the column with keys.
### Composite Key ### Composite Key
@ -93,7 +93,7 @@ Configuration example:
... ...
<attribute> <attribute>
<name>Name</name> <name>Name</name>
<type>Type</type> <type>ClickHouseDataType</type>
<null_value></null_value> <null_value></null_value>
<expression>rand64()</expression> <expression>rand64()</expression>
<hierarchical>true</hierarchical> <hierarchical>true</hierarchical>
@ -108,11 +108,11 @@ Configuration fields:
Tag | Description | Required Tag | Description | Required
----|-------------|--------- ----|-------------|---------
`name`| Column name. | Yes `name`| Column name. | Yes
`type`| Column type.<br/>Sets the method for interpreting data in the source. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the source table, but it can be uploaded as `String`. | Yes `type`| ClickHouse data type.<br/>ClickHouse tries to cast value from dictionary to the specified data type. For example, for MySQL, the field might be `TEXT`, `VARCHAR`, or `BLOB` in the MySQL source table, but it can be uploaded as `String` in ClickHouse.<br/>[Nullable](../../data_types/nullable.md) is not supported. | Yes
`null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. | Yes `null_value` | Default value for a non-existing element.<br/>In the example, it is an empty string. You cannot use `NULL` in this field. | Yes
`expression` | [Expression](../syntax.md#syntax-expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it for creating an alias for the remote column.<br/><br/>Default value: no expression. | No `expression` | [Expression](../syntax.md#syntax-expressions) that ClickHouse executes on the value.<br/>The expression can be a column name in the remote SQL database. Thus, you can use it to create an alias for the remote column.<br/><br/>Default value: no expression. | No
`hierarchical` | Hierarchical support. Mirrored to the parent identifier.<br/><br/>Default value: `false`. | No `hierarchical` | Hierarchical support. Mirrored to the parent identifier.<br/><br/>Default value: `false`. | No
`injective` | Flag that shows whether the `id -> attribute` image is injective.<br/>If `true`, then you can optimize the `GROUP BY` clause.<br/><br/>Default value: `false`. | No `injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).<br/>If `true`, ClickHouse can automatically place after the `GROUP BY` clause the requests to dictionaries with injection. Usually it significantly reduces the amount of such requests.<br/><br/>Default value: `false`. | No
`is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.<br/><br/>Default value: `false`. | No `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.<br/><br/>Default value: `false`. | No
[Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_structure/) <!--hide--> [Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_structure/) <!--hide-->

View File

@ -1,16 +1,16 @@
# Hash functions # Hash functions
Hash functions can be used for deterministic pseudo-random shuffling of elements. Hash functions can be used for the deterministic pseudo-random shuffling of elements.
## halfMD5 {#hash_functions-halfmd5} ## halfMD5 {#hash_functions-halfmd5}
[Interprets](../../query_language/functions/type_conversion_functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the MD5 hash value for each of them. Then combines hashes. Then from the resulting string, takes the first 8 bytes of the hash and interprets them as `UInt64` in big-endian byte order. [Interprets](../../query_language/functions/type_conversion_functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the [MD5](https://en.wikipedia.org/wiki/MD5) hash value for each of them. Then combines hashes, takes the first 8 bytes of the hash of the resulting string, and interprets them as `UInt64` in big-endian byte order.
``` ```
halfMD5(par1, ...) halfMD5(par1, ...)
``` ```
The function works relatively slow (5 million short strings per second per processor core). The function is relatively slow (5 million short strings per second per processor core).
Consider using the [sipHash64](#hash_functions-siphash64) function instead. Consider using the [sipHash64](#hash_functions-siphash64) function instead.
**Parameters** **Parameters**
@ -19,7 +19,7 @@ The function takes a variable number of input parameters. Parameters can be any
**Returned Value** **Returned Value**
Hash value having the [UInt64](../../data_types/int_uint.md) data type. A [UInt64](../../data_types/int_uint.md) data type hash value.
**Example** **Example**
@ -40,23 +40,28 @@ If you want to get the same result as output by the md5sum utility, use lower(he
## sipHash64 {#hash_functions-siphash64} ## sipHash64 {#hash_functions-siphash64}
Produces 64-bit [SipHash](https://131002.net/siphash/) hash value. Produces a 64-bit [SipHash](https://131002.net/siphash/) hash value.
``` ```
sipHash64(par1,...) sipHash64(par1,...)
``` ```
This function [interprets](../../query_language/functions/type_conversion_functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes.
This is a cryptographic hash function. It works at least three times faster than the [MD5](#hash_functions-md5) function. This is a cryptographic hash function. It works at least three times faster than the [MD5](#hash_functions-md5) function.
Function [interprets](../../query_language/functions/type_conversion_functions.md#type_conversion_functions-reinterpretAsString) all the input parameters as strings and calculates the hash value for each of them. Then combines hashes by the following algorithm:
1. After hashing all the input parameters, the function gets the array of hashes.
2. Function takes the first and the second elements and calculates a hash for the array of them.
3. Then the function takes the hash value, calculated at the previous step, and the third element of the initial hash array, and calculates a hash for the array of them.
4. The previous step is repeated for all the remaining elements of the initial hash array.
**Parameters** **Parameters**
The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../data_types/index.md). The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../data_types/index.md).
**Returned Value** **Returned Value**
Hash value having the [UInt64](../../data_types/int_uint.md) data type. A [UInt64](../../data_types/int_uint.md) data type hash value.
**Example** **Example**
@ -77,13 +82,13 @@ Differs from sipHash64 in that the final xor-folding state is only done up to 12
## cityHash64 ## cityHash64
Produces 64-bit hash value. Produces a 64-bit [CityHash](https://github.com/google/cityhash) hash value.
``` ```
cityHash64(par1,...) cityHash64(par1,...)
``` ```
This is the fast non-cryptographic hash function. It uses [CityHash](https://github.com/google/cityhash) algorithm for string parameters and implementation-specific fast non-cryptographic hash function for the parameters with other data types. To get the final result, the function uses the CityHash combinator. This is a fast non-cryptographic hash function. It uses the CityHash algorithm for string parameters and implementation-specific fast non-cryptographic hash function for parameters with other data types. The function uses the CityHash combinator to get the final results.
**Parameters** **Parameters**
@ -91,7 +96,7 @@ The function takes a variable number of input parameters. Parameters can be any
**Returned Value** **Returned Value**
Hash value having the [UInt64](../../data_types/int_uint.md) data type. A [UInt64](../../data_types/int_uint.md) data type hash value.
**Examples** **Examples**
@ -100,7 +105,7 @@ Call example:
```sql ```sql
SELECT cityHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS CityHash, toTypeName(CityHash) AS type SELECT cityHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS CityHash, toTypeName(CityHash) AS type
``` ```
``` ```text
┌─────────────CityHash─┬─type───┐ ┌─────────────CityHash─┬─type───┐
│ 12072650598913549138 │ UInt64 │ │ 12072650598913549138 │ UInt64 │
└──────────────────────┴────────┘ └──────────────────────┴────────┘
@ -108,8 +113,8 @@ SELECT cityHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:0
The following example shows how to compute the checksum of the entire table with accuracy up to the row order: The following example shows how to compute the checksum of the entire table with accuracy up to the row order:
``` ```sql
SELECT sum(cityHash64(*)) FROM table SELECT groupBitXor(cityHash64(*)) FROM table
``` ```
@ -157,7 +162,7 @@ The function takes a variable number of input parameters. Parameters can be any
**Returned Value** **Returned Value**
Hash value having the [UInt64](../../data_types/int_uint.md) data type. A [UInt64](../../data_types/int_uint.md) data type hash value.
**Example** **Example**
@ -172,15 +177,15 @@ SELECT farmHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:0
## javaHash {#hash_functions-javahash} ## javaHash {#hash_functions-javahash}
Calculates JavaHash from a string. Calculates [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452)
from a string.
Accepts a String-type argument. Returns Int32. Accepts a String-type argument. Returns Int32.
For more information, see the link: [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452)
## hiveHash ## hiveHash
Calculates HiveHash from a string. Calculates HiveHash from a string.
Accepts a String-type argument. Returns Int32. Accepts a String-type argument. Returns Int32.
Same as for [JavaHash](#hash_functions-javahash), except that the return value never has a negative number. This is just [JavaHash](#hash_functions-javahash) with zeroed out sign bit. This function is used in [Apache Hive](https://en.wikipedia.org/wiki/Apache_Hive) for versions before 3.0.
## metroHash64 ## metroHash64
@ -196,7 +201,7 @@ The function takes a variable number of input parameters. Parameters can be any
**Returned Value** **Returned Value**
Hash value having the [UInt64](../../data_types/int_uint.md) data type. A [UInt64](../../data_types/int_uint.md) data type hash value.
**Example** **Example**
@ -259,8 +264,8 @@ Both functions take a variable number of input parameters. Parameters can be any
**Returned Value** **Returned Value**
- The `murmurHash3_32` function returns hash value having the [UInt32](../../data_types/int_uint.md) data type. - The `murmurHash3_32` function returns a [UInt32](../../data_types/int_uint.md) data type hash value.
- The `murmurHash3_64` function returns hash value having the [UInt64](../../data_types/int_uint.md) data type. - The `murmurHash3_64` function returns a [UInt64](../../data_types/int_uint.md) data type hash value.
**Example** **Example**
@ -283,11 +288,11 @@ murmurHash3_128( expr )
**Parameters** **Parameters**
- `expr` — [Expressions](../syntax.md#syntax-expressions) returning [String](../../data_types/string.md)-typed value. - `expr` — [Expressions](../syntax.md#syntax-expressions) returning a [String](../../data_types/string.md)-type value.
**Returned Value** **Returned Value**
Hash value having [FixedString(16) data type](../../data_types/fixedstring.md). A [FixedString(16)](../../data_types/fixedstring.md) data type hash value.
**Example** **Example**

View File

@ -39,9 +39,11 @@ The `CHECK TABLE` query supports the following table engines:
- [StripeLog](../operations/table_engines/stripelog.md) - [StripeLog](../operations/table_engines/stripelog.md)
- [MergeTree family](../operations/table_engines/mergetree.md) - [MergeTree family](../operations/table_engines/mergetree.md)
The `*Log` engines do not provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner. Performed over the tables with another table engines causes an exception.
For the `MergeTree` family engines the `CHECK TABLE` query shows a check status for every individual table data part at the local server. Engines from the `*Log` family don't provide automatic data recovery on failure. Use the `CHECK TABLE` query to track data loss in a timely manner.
For `MergeTree` family engines, the `CHECK TABLE` query shows a check status for every individual data part of a table on the local server.
**If the data is corrupted** **If the data is corrupted**

View File

@ -31,7 +31,7 @@ ClickHouse применяет настройку в тех случаях, ко
- 0 — выключена. - 0 — выключена.
- 1 — включена. - 1 — включена.
Значение по умолчанию: 0. Значение по умолчанию: 1.
**Использование** **Использование**

View File

@ -1,4 +1,3 @@
# Ключ и поля словаря # Ключ и поля словаря
Секция `<structure>` описывает ключ словаря и поля, доступные для запросов. Секция `<structure>` описывает ключ словаря и поля, доступные для запросов.
@ -24,25 +23,24 @@
В структуре описываются столбцы: В структуре описываются столбцы:
- `<id>` - [ключевой столбец](external_dicts_dict_structure.md). - `<id>` [ключевой столбец](external_dicts_dict_structure.md#ext_dict_structure-key).
- `<attribute>` - [столбец данных](external_dicts_dict_structure.md). Столбцов может быть много. - `<attribute>` [столбец данных](external_dicts_dict_structure.md#ext_dict_structure-attributes). Столбцов может быть много.
## Ключ {#ext_dict_structure-key}
## Ключ
ClickHouse поддерживает следующие виды ключей: ClickHouse поддерживает следующие виды ключей:
- Числовой ключ. Формат UInt64. Описывается в теге `<id>`. - Числовой ключ. UInt64. Описывается в теге `<id>`.
- Составной ключ. Набор значений разного типа. Описывается в теге `<key>`. - Составной ключ. Набор значений разного типа. Описывается в теге `<key>`.
Структура может содержать либо `<id>` либо `<key>`. Структура может содержать либо `<id>` либо `<key>`.
!!! attention "Обратите внимание" !!! warning "Обратите внимание"
Ключ не надо дополнительно описывать в атрибутах. Ключ не надо дополнительно описывать в атрибутах.
### Числовой ключ ### Числовой ключ
Формат: `UInt64`. Тип: `UInt64`.
Пример конфигурации: Пример конфигурации:
@ -54,11 +52,11 @@ ClickHouse поддерживает следующие виды ключей:
Поля конфигурации: Поля конфигурации:
- name - имя столбца с ключами. - `name` имя столбца с ключами.
### Составной ключ ### Составной ключ
Ключом может быть кортеж (`tuple`) из полей произвольных типов. [layout](external_dicts_dict_layout.md) в этом случае должен быть `complex_key_hashed` или `complex_key_cache`. Ключoм может быть кортеж (`tuple`) из полей произвольных типов. В этом случае [layout](external_dicts_dict_layout.md) должен быть `complex_key_hashed` или `complex_key_cache`.
!!! tip "Совет" !!! tip "Совет"
Cоставной ключ может состоять из одного элемента. Это даёт возможность использовать в качестве ключа, например, строку. Cоставной ключ может состоять из одного элемента. Это даёт возможность использовать в качестве ключа, например, строку.
@ -93,7 +91,7 @@ ClickHouse поддерживает следующие виды ключей:
... ...
<attribute> <attribute>
<name>Name</name> <name>Name</name>
<type>Type</type> <type>ClickHouseDataType</type>
<null_value></null_value> <null_value></null_value>
<expression>rand64()</expression> <expression>rand64()</expression>
<hierarchical>true</hierarchical> <hierarchical>true</hierarchical>
@ -105,13 +103,14 @@ ClickHouse поддерживает следующие виды ключей:
Поля конфигурации: Поля конфигурации:
- `name` - Имя столбца. | Тег | Описание | Обязательный |
- `type` - Тип столбца. Задает способ интерпретации данных в источнике. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. | ---- | ------------- | --------- |
- `null_value` - Значение по умолчанию для несуществующего элемента. В примере - пустая строка. | `name` | Имя столбца. | Да |
- `expression` - Атрибут может быть выражением. Тег не обязательный. | `type` | Тип данных ClickHouse.<br/>ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. [Nullable](../../data_types/nullable.md) не поддерживается. | Да |
- `hierarchical` - Поддержка иерархии. Отображение в идентификатор родителя. По умолчанию, `false`. | `null_value` | Значение по умолчанию для несуществующего элемента.<br/>В примере это пустая строка. Нельзя указать значение `NULL`. | Да |
- `injective` - Признак инъективности отображения `id -> attribute`. Если `true`, то можно оптимизировать `GROUP BY`. По умолчанию, `false`. | `expression` | [Выражение](../syntax.md#syntax-expressions), которое ClickHouse выполняет со значением.<br/>Выражением может быть имя столбца в удаленной SQL базе. Таким образом, вы можете использовать его для создания псевдонима удаленного столбца.<br/><br/>Значение по умолчанию: нет выражения. | Нет |
- `is_object_id` - Признак того, что запрос выполняется к документу MongoDB по `ObjectID`. | `hierarchical` | Поддержка иерархии. Отображение в идентификатор родителя.<br/><br/>Значение по умолчанию: `false`. | Нет |
| `injective` | Признак [инъективности](https://ru.wikipedia.org/wiki/Инъекция_(математика)) отображения `id -> attribute`. <br/>Если `true`, то обращения к словарям с включенной инъективностью могут быть автоматически переставлены ClickHouse за стадию `GROUP BY`, что как правило существенно сокращает их количество.<br/><br/>Значение по умолчанию: `false`. | Нет |
| `is_object_id` | Признак того, что запрос выполняется к документу MongoDB по `ObjectID`.<br/><br/>Значение по умолчанию: `false`. | Нет |
[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_structure/) <!--hide--> [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_structure/) <!--hide-->

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