diff --git a/contrib/librdkafka-cmake/CMakeLists.txt b/contrib/librdkafka-cmake/CMakeLists.txt index d060994ef05..75cd3968204 100644 --- a/contrib/librdkafka-cmake/CMakeLists.txt +++ b/contrib/librdkafka-cmake/CMakeLists.txt @@ -7,6 +7,8 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdavl.c ${RDKAFKA_SOURCE_DIR}/rdbuf.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_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_background.c @@ -28,11 +30,13 @@ set(SRCS ${RDKAFKA_SOURCE_DIR}/rdkafka_op.c ${RDKAFKA_SOURCE_DIR}/rdkafka_partition.c ${RDKAFKA_SOURCE_DIR}/rdkafka_pattern.c + ${RDKAFKA_SOURCE_DIR}/rdkafka_plugin.c ${RDKAFKA_SOURCE_DIR}/rdkafka_queue.c ${RDKAFKA_SOURCE_DIR}/rdkafka_range_assignor.c ${RDKAFKA_SOURCE_DIR}/rdkafka_request.c ${RDKAFKA_SOURCE_DIR}/rdkafka_roundrobin_assignor.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_scram.c ${RDKAFKA_SOURCE_DIR}/rdkafka_ssl.c diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h index bf67863ae7d..78b9bc613c3 100644 --- a/contrib/librdkafka-cmake/config.h +++ b/contrib/librdkafka-cmake/config.h @@ -16,7 +16,7 @@ #define MKL_APP_NAME "librdkafka" #define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library" // distro -//#define SOLIB_EXT ".so" +#define SOLIB_EXT ".so" // gcc //#define WITH_GCC 1 // gxx @@ -48,9 +48,9 @@ // parseversion #define MKL_APP_VERSION "0.11.4" // libdl -//#define WITH_LIBDL 1 +#define WITH_LIBDL 1 // WITH_PLUGINS -//#define WITH_PLUGINS 1 +#define WITH_PLUGINS 1 // zlib #define WITH_ZLIB 1 // zstd @@ -63,6 +63,8 @@ #define WITH_SSL 1 // WITH_SASL_SCRAM #define WITH_SASL_SCRAM 1 +// WITH_SASL_OAUTHBEARER +#define WITH_SASL_OAUTHBEARER 1 // crc32chw #if !defined(__PPC__) #define WITH_CRC32C_HW 1 @@ -78,5 +80,5 @@ // python //#define HAVE_PYTHON 1 // disable C11 threads for compatibility with old libc -#define WITH_C11THREADS 0 +//#define WITH_C11THREADS 1 #endif /* _CONFIG_H_ */ diff --git a/contrib/zlib-ng b/contrib/zlib-ng index 9173b89d467..cb43e7fa08e 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit 9173b89d46799582d20a30578e0aa9788bc7d6e1 +Subproject commit cb43e7fa08ec29fd76d84e3bb35258a0f0bf3df3 diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index afb0ad3d6c2..57821d854e9 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -20,6 +20,9 @@ #include +#include + + /// Universal executable for various clickhouse applications #if ENABLE_CLICKHOUSE_SERVER || !defined(ENABLE_CLICKHOUSE_SERVER) int mainEntryClickHouseServer(int argc, char ** argv); @@ -144,6 +147,11 @@ int main(int argc_, char ** argv_) /// It is needed because LLVM library clobbers it. 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 (argc_ >= 2 && 0 == strcmp(argv_[1], "-cc1")) return mainEntryClickHouseClang(argc_, argv_); diff --git a/dbms/programs/server/Server.cpp b/dbms/programs/server/Server.cpp index 58f15f878b4..108b2aeee20 100644 --- a/dbms/programs/server/Server.cpp +++ b/dbms/programs/server/Server.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -509,7 +510,8 @@ int Server::main(const std::vector & /*args*/) LOG_DEBUG(log, "Loaded metadata."); /// Init trace collector only after trace_log system table was created - global_context->initializeTraceCollector(); + if (hasPHDRCache()) + global_context->initializeTraceCollector(); global_context->setCurrentDatabase(default_database); diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 04c8bf15c03..acf58abfbb4 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -210,7 +210,13 @@ void ConfigProcessor::mergeRecursive(XMLDocumentPtr config, Node * config_root, 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() diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index c472a336d73..e8ee16c5670 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -437,6 +437,7 @@ namespace ErrorCodes extern const int CANNOT_CREATE_TIMER = 460; extern const int CANNOT_SET_TIMER_PERIOD = 461; extern const int CANNOT_DELETE_TIMER = 462; + extern const int CANNOT_FCNTL = 463; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 7059e02d76c..277aafa9eb8 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -171,6 +171,9 @@ M(OSReadChars, "Number of bytes read from 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(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 { diff --git a/dbms/src/Common/QueryProfiler.cpp b/dbms/src/Common/QueryProfiler.cpp index 5fd1c181994..546fe78fbc9 100644 --- a/dbms/src/Common/QueryProfiler.cpp +++ b/dbms/src/Common/QueryProfiler.cpp @@ -1,14 +1,26 @@ #include "QueryProfiler.h" +#include +#include #include +#include +#include #include #include #include #include #include +#include #include #include + +namespace ProfileEvents +{ + extern const Event QueryProfilerCannotWriteTrace; + extern const Event QueryProfilerSignalOverruns; +} + namespace DB { @@ -16,23 +28,76 @@ extern LazyPipe trace_pipe; 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. /// 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; - 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 8 * sizeof(char) + // maximum VarUInt length for string size QUERY_ID_MAX_LEN * sizeof(char) + // maximum query_id length sizeof(StackTrace) + // collected stack trace - sizeof(TimerType); // timer type + sizeof(TimerType) + // timer type + sizeof(UInt32); // thread_number 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(); 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(context); const StackTrace stack_trace(signal_context); @@ -40,6 +105,7 @@ namespace writeStringBinary(query_id, out); writePODBinary(stack_trace, out); writePODBinary(timer_type, out); + writePODBinary(thread_number, out); out.next(); } @@ -56,10 +122,19 @@ namespace ErrorCodes } template -QueryProfilerBase::QueryProfilerBase(const Int32 thread_id, const int clock_type, const UInt32 period, const int pause_signal) +QueryProfilerBase::QueryProfilerBase(const Int32 thread_id, const int clock_type, UInt32 period, const int pause_signal) : log(&Logger::get("QueryProfiler")) , 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{}; sa.sa_sigaction = ProfilerImpl::signalHandler; sa.sa_flags = SA_SIGINFO | SA_RESTART; @@ -87,8 +162,16 @@ QueryProfilerBase::QueryProfilerBase(const Int32 thread_id, const if (timer_create(clock_type, &sev, &timer_id)) 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(0, period)(rng); + 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)) throwFromErrno("Failed to set thread timer period", ErrorCodes::CANNOT_SET_TIMER_PERIOD); } @@ -97,6 +180,9 @@ QueryProfilerBase::QueryProfilerBase(const Int32 thread_id, const tryCleanup(); throw; } +#else + throw Exception("QueryProfiler cannot work with stock libunwind", ErrorCodes::NOT_IMPLEMENTED); +#endif } template diff --git a/dbms/src/Common/QueryProfiler.h b/dbms/src/Common/QueryProfiler.h index d4e92f25a17..5eaf5e2e7f7 100644 --- a/dbms/src/Common/QueryProfiler.h +++ b/dbms/src/Common/QueryProfiler.h @@ -5,6 +5,7 @@ #include #include + namespace Poco { class Logger; @@ -34,8 +35,7 @@ template class QueryProfilerBase { 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(); private: diff --git a/dbms/src/Common/SharedLibrary.cpp b/dbms/src/Common/SharedLibrary.cpp index 30ed3bccaab..568bfaa4f3e 100644 --- a/dbms/src/Common/SharedLibrary.cpp +++ b/dbms/src/Common/SharedLibrary.cpp @@ -1,6 +1,7 @@ #include "SharedLibrary.h" #include #include +#include #include "Exception.h" @@ -17,6 +18,8 @@ SharedLibrary::SharedLibrary(const std::string & path, int flags) handle = dlopen(path.c_str(), flags); if (!handle) throw Exception(std::string("Cannot dlopen: ") + dlerror(), ErrorCodes::CANNOT_DLOPEN); + + updatePHDRCache(); } SharedLibrary::~SharedLibrary() diff --git a/dbms/src/Common/TraceCollector.cpp b/dbms/src/Common/TraceCollector.cpp index 293e4c38e97..e66a580289d 100644 --- a/dbms/src/Common/TraceCollector.cpp +++ b/dbms/src/Common/TraceCollector.cpp @@ -12,6 +12,10 @@ #include #include +#include +#include + + namespace DB { @@ -21,6 +25,7 @@ namespace ErrorCodes { extern const int NULL_POINTER_DEREFERENCE; extern const int THREAD_IS_NOT_JOINABLE; + extern const int CANNOT_FCNTL; } TraceCollector::TraceCollector(std::shared_ptr & trace_log) @@ -31,6 +36,28 @@ TraceCollector::TraceCollector(std::shared_ptr & trace_log) throw Exception("Invalid trace log pointer passed", ErrorCodes::NULL_POINTER_DEREFERENCE); 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); } @@ -78,10 +105,12 @@ void TraceCollector::run() std::string query_id; StackTrace stack_trace(NoCapture{}); TimerType timer_type; + UInt32 thread_number; readStringBinary(query_id, in); readPODBinary(stack_trace, in); readPODBinary(timer_type, in); + readPODBinary(thread_number, in); const auto size = stack_trace.getSize(); const auto & frames = stack_trace.getFrames(); @@ -91,7 +120,7 @@ void TraceCollector::run() for (size_t i = 0; i < size; i++) trace.emplace_back(UInt64(reinterpret_cast(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); } diff --git a/dbms/src/Common/memcmpSmall.h b/dbms/src/Common/memcmpSmall.h index 13269a13b29..5dc5e04a707 100644 --- a/dbms/src/Common/memcmpSmall.h +++ b/dbms/src/Common/memcmpSmall.h @@ -3,8 +3,7 @@ #include #include -#ifdef __SSE2__ -#include +#include 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 + + /** All functions works under the following assumptions: * - 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. @@ -192,7 +200,10 @@ inline bool memoryIsZeroSmallAllowOverflow15(const void * data, size_t size) template 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 diff --git a/dbms/src/Common/new_delete.cpp b/dbms/src/Common/new_delete.cpp index aff708135e1..9da6ccf492f 100644 --- a/dbms/src/Common/new_delete.cpp +++ b/dbms/src/Common/new_delete.cpp @@ -7,7 +7,7 @@ /// Replace default new/delete with memory tracking versions. /// @sa https://en.cppreference.com/w/cpp/memory/new/operator_new /// https://en.cppreference.com/w/cpp/memory/new/operator_delete -#if NOT_UNBUNDLED +#if !UNBUNDLED namespace Memory { diff --git a/dbms/src/Core/MySQLProtocol.h b/dbms/src/Core/MySQLProtocol.h index 3d908971601..d06949285fa 100644 --- a/dbms/src/Core/MySQLProtocol.h +++ b/dbms/src/Core/MySQLProtocol.h @@ -34,6 +34,7 @@ const size_t SSL_REQUEST_PAYLOAD_SIZE = 32; 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. } @@ -285,7 +286,12 @@ public: result.append(1, auth_plugin_data.size()); 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(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); return result; } diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 27e06ffec10..953a80b65ab 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -221,8 +221,8 @@ struct Settings : public SettingsCollection 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(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_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_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, "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. \ @@ -307,6 +307,7 @@ struct Settings : public SettingsCollection 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(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(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 \ /** 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) diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 82de731f4ad..7b5de251a0d 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -462,7 +462,7 @@ void DataTypeArray::deserializeProtobuf(IColumn & column, ProtobufReader & proto bool nested_row_added; do 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) { offsets.emplace_back(nested_column.size()); diff --git a/dbms/src/Formats/ProtobufReader.cpp b/dbms/src/Formats/ProtobufReader.cpp index ac45c1d94ee..5ef39315f5f 100644 --- a/dbms/src/Formats/ProtobufReader.cpp +++ b/dbms/src/Formats/ProtobufReader.cpp @@ -34,19 +34,18 @@ namespace BITS32 = 5, }; - // The following should be always true: - // REACHED_END < any cursor position < min(END_OF_VARINT, END_OF_GROUP) + // The following condition must always be true: + // any_cursor_position < min(END_OF_VARINT, END_OF_GROUP) // This inequation helps to check conditions in SimpleReader. constexpr UInt64 END_OF_VARINT = static_cast(-1); constexpr UInt64 END_OF_GROUP = static_cast(-2); Int64 decodeZigZag(UInt64 n) { return static_cast((n >> 1) ^ (~(n & 1) + 1)); } -} - -[[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); + [[noreturn]] void 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); + } } @@ -54,94 +53,103 @@ namespace // Knows nothing about protobuf schemas, just provides useful functions to deserialize data. ProtobufReader::SimpleReader::SimpleReader(ReadBuffer & in_) : in(in_) - , cursor(1 /* We starts at cursor == 1 to keep any cursor value > REACHED_END, this allows to simplify conditions */) - , current_message_end(REACHED_END) - , field_end(REACHED_END) + , cursor(0) + , current_message_level(0) + , current_message_end(0) + , field_end(0) + , last_string_pos(-1) { } bool ProtobufReader::SimpleReader::startMessage() { - if ((current_message_end == REACHED_END) && parent_message_ends.empty()) - { - // Start reading a root message. - if (unlikely(in.eof())) - return false; - size_t size_of_message = readVarint(); - if (size_of_message == 0) - throwUnknownFormat(); - 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; + // Start reading a root message. + assert(!current_message_level); + if (unlikely(in.eof())) + return false; + size_t size_of_message = readVarint(); + current_message_end = cursor + size_of_message; + ++current_message_level; + field_end = cursor; 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) - ignoreGroup(); - else if (cursor < current_message_end) - ignore(current_message_end - cursor); - else if (unlikely(cursor > current_message_end)) - { - if (!parent_message_ends.empty()) - throwUnknownFormat(); - moveCursorBackward(cursor - current_message_end); - } - current_message_end = REACHED_END; + if (cursor < root_message_end) + ignore(root_message_end - cursor); + else if (ignore_errors) + moveCursorBackward(cursor - root_message_end); + else + throwUnknownFormat(); } - field_end = REACHED_END; - if (!parent_message_ends.empty()) - { - current_message_end = parent_message_ends.back(); - parent_message_ends.pop_back(); - } + current_message_level = 0; + parent_message_ends.clear(); } -void ProtobufReader::SimpleReader::endRootMessage() +void ProtobufReader::SimpleReader::startNestedMessage() { - UInt64 message_end = parent_message_ends.empty() ? current_message_end : parent_message_ends.front(); - if (message_end != REACHED_END) + assert(current_message_level >= 1); + // 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) - ignore(message_end - cursor); - else if (unlikely(cursor > message_end)) - moveCursorBackward(cursor - message_end); + if (current_message_end == END_OF_GROUP) + { + ignoreGroup(); + 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; - field_end = REACHED_END; + + --current_message_level; + current_message_end = parent_message_ends.back(); + parent_message_ends.pop_back(); + field_end = cursor; } 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) + { ignoreVarint(); + field_end = cursor; + } else if (field_end == END_OF_GROUP) + { ignoreGroup(); + field_end = cursor; + } else if (cursor < field_end) ignore(field_end - cursor); - field_end = REACHED_END; + else + throwUnknownFormat(); } if (cursor >= current_message_end) - { - current_message_end = REACHED_END; return false; - } UInt64 varint = readVarint(); if (unlikely(varint & (static_cast(0xFFFFFFFF) << 32))) @@ -151,6 +159,11 @@ bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number) WireType wire_type = static_cast(key & 0x07); switch (wire_type) { + case BITS32: + { + field_end = cursor + 4; + return true; + } case BITS64: { field_end = cursor + 8; @@ -176,29 +189,20 @@ bool ProtobufReader::SimpleReader::readFieldNumber(UInt32 & field_number) { if (current_message_end != END_OF_GROUP) throwUnknownFormat(); - current_message_end = REACHED_END; + current_message_end = cursor; return false; } - case BITS32: - { - field_end = cursor + 4; - return true; - } } throwUnknownFormat(); - __builtin_unreachable(); } bool ProtobufReader::SimpleReader::readUInt(UInt64 & value) { if (unlikely(cursor >= field_end)) - { - field_end = REACHED_END; return false; - } value = readVarint(); - if ((field_end == END_OF_VARINT) || (cursor >= field_end)) - field_end = REACHED_END; + if (field_end == END_OF_VARINT) + field_end = cursor; return true; } @@ -224,25 +228,22 @@ template bool ProtobufReader::SimpleReader::readFixed(T & value) { if (unlikely(cursor >= field_end)) - { - field_end = REACHED_END; return false; - } readBinary(&value, sizeof(T)); - if (cursor >= field_end) - field_end = REACHED_END; return true; } bool ProtobufReader::SimpleReader::readStringInto(PaddedPODArray & 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)) - return false; + throwUnknownFormat(); size_t length = field_end - cursor; size_t old_size = str.size(); str.resize(old_size + length); readBinary(reinterpret_cast(str.data() + old_size), length); - field_end = REACHED_END; return true; } @@ -299,7 +300,6 @@ UInt64 ProtobufReader::SimpleReader::continueReadingVarint(UInt64 first_byte) #undef PROTOBUF_READER_READ_VARINT_BYTE throwUnknownFormat(); - __builtin_unreachable(); } void ProtobufReader::SimpleReader::ignoreVarint() @@ -1083,9 +1083,9 @@ bool ProtobufReader::startMessage() 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_converter = nullptr; } @@ -1102,7 +1102,7 @@ bool ProtobufReader::readColumnIndex(size_t & column_index) current_converter = nullptr; return false; } - simple_reader.endMessage(); + simple_reader.endNestedMessage(); current_field_index = current_message->index_in_parent; current_message = current_message->parent; continue; @@ -1132,7 +1132,7 @@ bool ProtobufReader::readColumnIndex(size_t & column_index) if (field->nested_message) { - simple_reader.startMessage(); + simple_reader.startNestedMessage(); current_message = field->nested_message.get(); current_field_index = 0; continue; diff --git a/dbms/src/Formats/ProtobufReader.h b/dbms/src/Formats/ProtobufReader.h index 9c5fdfb5fe4..c2660369c67 100644 --- a/dbms/src/Formats/ProtobufReader.h +++ b/dbms/src/Formats/ProtobufReader.h @@ -42,7 +42,7 @@ public: bool startMessage(); /// Ends reading a message. - void endMessage(); + void endMessage(bool ignore_errors = false); /// Reads the column index. /// 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); } - /// When it returns false there is no more values left and from now on all the read() functions will return false - /// until readColumnIndex() is called. When it returns true it's unclear. - bool ALWAYS_INLINE maybeCanReadValue() const { return simple_reader.maybeCanReadValue(); } + /// Call it after calling one of the read*() function to determine if there are more values available for reading. + bool ALWAYS_INLINE canReadMoreValues() const { return simple_reader.canReadMoreValues(); } private: class SimpleReader @@ -89,8 +88,9 @@ private: public: SimpleReader(ReadBuffer & in_); bool startMessage(); - void endMessage(); - void endRootMessage(); + void endMessage(bool ignore_errors); + void startNestedMessage(); + void endNestedMessage(); bool readFieldNumber(UInt32 & field_number); bool readInt(Int64 & value); bool readSInt(Int64 & value); @@ -98,15 +98,7 @@ private: template bool readFixed(T & value); bool readStringInto(PaddedPODArray & str); - bool ALWAYS_INLINE maybeCanReadValue() const - { - if (field_end == REACHED_END) - return false; - if (cursor < root_message_end) - return true; - - throwUnknownFormat(); - } + bool ALWAYS_INLINE canReadMoreValues() const { return cursor < field_end; } private: void readBinary(void * data, size_t size); @@ -128,17 +120,13 @@ private: void ignoreVarint(); void ignoreGroup(); - [[noreturn]] static void throwUnknownFormat(); - - static constexpr UInt64 REACHED_END = 0; - ReadBuffer & in; UInt64 cursor; - std::vector parent_message_ends; + size_t current_message_level; UInt64 current_message_end; + std::vector parent_message_ends; UInt64 field_end; - - UInt64 root_message_end; + UInt64 last_string_pos; }; class IConverter diff --git a/dbms/src/Formats/ProtobufRowInputStream.cpp b/dbms/src/Formats/ProtobufRowInputStream.cpp index ff5717c1bca..8dcff7bda36 100644 --- a/dbms/src/Formats/ProtobufRowInputStream.cpp +++ b/dbms/src/Formats/ProtobufRowInputStream.cpp @@ -41,7 +41,7 @@ bool ProtobufRowInputStream::read(MutableColumns & columns, RowReadExtension & e read_columns[column_index] = true; allow_add_row = false; } - } while (reader.maybeCanReadValue()); + } while (reader.canReadMoreValues()); } // Fill non-visited columns with the default values. @@ -60,7 +60,7 @@ bool ProtobufRowInputStream::allowSyncAfterError() const void ProtobufRowInputStream::syncAfterError() { - reader.endMessage(); + reader.endMessage(true); } diff --git a/dbms/src/Formats/ProtobufWriter.cpp b/dbms/src/Formats/ProtobufWriter.cpp index bca3449cb59..fcaedcab2a0 100644 --- a/dbms/src/Formats/ProtobufWriter.cpp +++ b/dbms/src/Formats/ProtobufWriter.cpp @@ -141,7 +141,8 @@ void ProtobufWriter::SimpleWriter::endMessage() size_t size_of_message = buffer.size() - num_bytes_skipped; writeVarint(size_of_message, out); for (const auto & piece : pieces) - out.write(reinterpret_cast(&buffer[piece.start]), piece.end - piece.start); + if (piece.end > piece.start) + out.write(reinterpret_cast(&buffer[piece.start]), piece.end - piece.start); buffer.clear(); pieces.clear(); num_bytes_skipped = 0; diff --git a/dbms/src/Functions/FunctionsIntrospection.cpp b/dbms/src/Functions/FunctionsIntrospection.cpp deleted file mode 100644 index 7dedc1daae3..00000000000 --- a/dbms/src/Functions/FunctionsIntrospection.cpp +++ /dev/null @@ -1,12 +0,0 @@ -#include -#include - -namespace DB -{ - -void registerFunctionsIntrospection(FunctionFactory & factory) -{ - factory.registerFunction(); -} - -} diff --git a/dbms/src/Functions/FunctionsIntrospection.h b/dbms/src/Functions/FunctionsIntrospection.h deleted file mode 100644 index d7ca1d37efa..00000000000 --- a/dbms/src/Functions/FunctionsIntrospection.h +++ /dev/null @@ -1,107 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include -#include -#include -#include -#include - -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(); - } - - 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(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(); - } - - 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(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 * data_vector = checkAndGetColumn>(&*data_ptr); - - const typename ColumnVector::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(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); - } -}; - -} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index a1c75aaf33a..178f085e1ad 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -40,6 +40,7 @@ void registerFunctionsIntrospection(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); void registerFunctionsJSON(FunctionFactory &); +void registerFunctionSymbolizeAddress(FunctionFactory &); void registerFunctions() { @@ -75,10 +76,10 @@ void registerFunctions() registerFunctionsVisitParam(factory); registerFunctionsMath(factory); registerFunctionsGeo(factory); - registerFunctionsIntrospection(factory); registerFunctionsNull(factory); registerFunctionsFindCluster(factory); registerFunctionsJSON(factory); + registerFunctionSymbolizeAddress(factory); } } diff --git a/dbms/src/Functions/symbolizeAddress.cpp b/dbms/src/Functions/symbolizeAddress.cpp new file mode 100644 index 00000000000..1096a8924b3 --- /dev/null +++ b/dbms/src/Functions/symbolizeAddress.cpp @@ -0,0 +1,115 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +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(); + } + + 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(); + } + + bool useDefaultImplementationForConstants() const override + { + return true; + } + + static std::string addressToSymbol(UInt64 uint_address) + { + void * addr = unalignedLoad(&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(column.get()); + + if (!column_concrete) + throw Exception("Illegal column " + column->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + + const typename ColumnVector::Container & data = column_concrete->getData(); + auto result_column = ColumnString::create(); + + static SimpleCache 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(); +} + +} diff --git a/dbms/src/IO/DelimitedReadBuffer.h b/dbms/src/IO/DelimitedReadBuffer.h index 1d4b07265ae..a2a781185ce 100644 --- a/dbms/src/IO/DelimitedReadBuffer.h +++ b/dbms/src/IO/DelimitedReadBuffer.h @@ -21,6 +21,11 @@ public: return typeid_cast(buffer.get()); } + void reset() + { + BufferBase::set(nullptr, 0, 0); + } + protected: // 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 diff --git a/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h new file mode 100644 index 00000000000..7dc3051167a --- /dev/null +++ b/dbms/src/Interpreters/CollectJoinOnKeysVisitor.h @@ -0,0 +1,196 @@ +#pragma once + +#include +#include + +#include +#include +#include + + +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; + + 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()) + visit(*func, ast, data); + } + + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) + { + if (auto * func = node->as()) + 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 & out) + { + if (const auto * ident = ast->as()) + { + 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 left_identifiers; + std::vector 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(); + 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 & 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; + +} diff --git a/dbms/src/Interpreters/InDepthNodeVisitor.h b/dbms/src/Interpreters/InDepthNodeVisitor.h index 7706a1fdd34..6ed19da2e94 100644 --- a/dbms/src/Interpreters/InDepthNodeVisitor.h +++ b/dbms/src/Interpreters/InDepthNodeVisitor.h @@ -10,19 +10,19 @@ namespace DB /// 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. -template -class InDepthNodeVisitor +template +class InDepthNodeVisitorTemplate { public: using Data = typename Matcher::Data; - InDepthNodeVisitor(Data & data_, std::ostream * ostr_ = nullptr) + InDepthNodeVisitorTemplate(Data & data_, std::ostream * ostr_ = nullptr) : data(data_), visit_depth(0), ostr(ostr_) {} - void visit(ASTPtr & ast) + void visit(T & ast) { DumpASTNode dump(*ast, ostr, visit_depth, typeid(Matcher).name()); @@ -40,7 +40,7 @@ private: size_t visit_depth; std::ostream * ostr; - void visitChildren(ASTPtr & ast) + void visitChildren(T & ast) { for (auto & child : ast->children) if (Matcher::needChildVisit(ast, child)) @@ -48,6 +48,12 @@ private: } }; +template +using InDepthNodeVisitor = InDepthNodeVisitorTemplate; + +template +using ConstInDepthNodeVisitor = InDepthNodeVisitorTemplate; + /// Simple matcher for one node type without complex traversal logic. template class OneTypeMatcher diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp index b6e6545e475..a97233d0798 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.cpp @@ -138,7 +138,10 @@ bool PredicateExpressionsOptimizer::allowPushDown( const std::vector & dependencies, 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; else { diff --git a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h index b7e3cd430fc..f9df113abf2 100644 --- a/dbms/src/Interpreters/PredicateExpressionsOptimizer.h +++ b/dbms/src/Interpreters/PredicateExpressionsOptimizer.h @@ -39,6 +39,7 @@ class PredicateExpressionsOptimizer /// for PredicateExpressionsOptimizer const bool enable_optimize_predicate_expression; + const bool enable_optimize_predicate_expression_to_final_subquery; const bool join_use_nulls; template @@ -47,6 +48,7 @@ class PredicateExpressionsOptimizer max_expanded_ast_elements(settings.max_expanded_ast_elements), count_distinct_implementation(settings.count_distinct_implementation), 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) {} }; diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.cpp b/dbms/src/Interpreters/QueryAliasesVisitor.cpp index f9257870583..98069396d81 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.cpp +++ b/dbms/src/Interpreters/QueryAliasesVisitor.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -41,12 +42,25 @@ void QueryAliasesMatcher::visit(ASTPtr & ast, Data & data) { if (auto * s = ast->as()) visit(*s, ast, data); + else if (auto * q = ast->as()) + visit(*q, ast, data); else if (auto * aj = ast->as()) visit(*aj, ast, data); else 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(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 /// (skip the expression list itself and its children). 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)); subquery.setAlias(alias); - subquery.prefer_alias_to_column_name = true; aliases[alias] = ast; } else visitOther(ast, data); + + subquery.prefer_alias_to_column_name = true; } void QueryAliasesMatcher::visitOther(const ASTPtr & ast, Data & data) diff --git a/dbms/src/Interpreters/QueryAliasesVisitor.h b/dbms/src/Interpreters/QueryAliasesVisitor.h index 458acce66f6..c4e297965c3 100644 --- a/dbms/src/Interpreters/QueryAliasesVisitor.h +++ b/dbms/src/Interpreters/QueryAliasesVisitor.h @@ -6,7 +6,7 @@ namespace DB { -class ASTSelectWithUnionQuery; +class ASTSelectQuery; class ASTSubquery; struct ASTTableExpression; struct ASTArrayJoin; @@ -26,6 +26,7 @@ public: static bool needChildVisit(ASTPtr & node, const ASTPtr & child); 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(const ASTArrayJoin &, const ASTPtr & ast, Data & data); static void visitOther(const ASTPtr & ast, Data & data); diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 83cacc94692..04102f5ae15 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include @@ -21,7 +22,6 @@ #include #include #include -#include #include @@ -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 get_table_belonging; - get_table_belonging = [&](const ASTPtr & ast) -> TableBelonging - { - if (IdentifierSemantic::getColumnName(ast)) - { - const auto * identifier = ast->as(); - - /// 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(); - 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(); - 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. -void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, - const NameSet & source_columns, const String & current_database, bool join_use_nulls) +void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, const NameSet & source_columns, + const Aliases & aliases, const String & current_database, bool join_use_nulls) { const ASTTablesInSelectQueryElement * node = select_query.join(); - if (!node) return; @@ -619,7 +504,17 @@ void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & s name = joined_table_name.getQualifiedNamePrefix() + name; } 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); @@ -771,7 +666,8 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Push the predicate expression down to the subqueries. 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(result); diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index c1fcf72667b..8c46a3ba08f 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -41,8 +41,6 @@ void ThreadStatus::attachQueryContext(Context & query_context_) if (!thread_group->global_context) thread_group->global_context = global_context; } - - initQueryProfiler(); } void CurrentThread::defaultThreadDeleter() @@ -124,6 +122,7 @@ void ThreadStatus::attachQuery(const ThreadGroupStatusPtr & thread_group_, bool #endif initPerformanceCounters(); + initQueryProfiler(); thread_state = ThreadState::AttachedToQuery; } @@ -155,7 +154,7 @@ void ThreadStatus::finalizePerformanceCounters() void ThreadStatus::initQueryProfiler() { /// query profilers are useless without trace collector - if (!global_context->hasTraceCollector()) + if (!global_context || !global_context->hasTraceCollector()) return; const auto & settings = query_context->getSettingsRef(); @@ -163,14 +162,12 @@ void ThreadStatus::initQueryProfiler() if (settings.query_profiler_real_time_period_ns > 0) query_profiler_real = std::make_unique( /* thread_id */ os_thread_id, - /* period */ static_cast(settings.query_profiler_real_time_period_ns) - ); + /* period */ static_cast(settings.query_profiler_real_time_period_ns)); if (settings.query_profiler_cpu_time_period_ns > 0) query_profiler_cpu = std::make_unique( /* thread_id */ os_thread_id, - /* period */ static_cast(settings.query_profiler_cpu_time_period_ns) - ); + /* period */ static_cast(settings.query_profiler_cpu_time_period_ns)); } void ThreadStatus::finalizeQueryProfiler() diff --git a/dbms/src/Interpreters/TraceLog.cpp b/dbms/src/Interpreters/TraceLog.cpp index c5583476e99..5cde9d30f0f 100644 --- a/dbms/src/Interpreters/TraceLog.cpp +++ b/dbms/src/Interpreters/TraceLog.cpp @@ -4,6 +4,8 @@ #include #include #include +#include + using namespace DB; @@ -20,13 +22,15 @@ Block TraceLogElement::createBlock() { {std::make_shared(), "event_date"}, {std::make_shared(), "event_time"}, + {std::make_shared(), "revision"}, {std::make_shared(timer_values), "timer_type"}, + {std::make_shared(), "thread_number"}, {std::make_shared(), "query_id"}, {std::make_shared(std::make_shared()), "trace"} }; } -void TraceLogElement::appendToBlock(Block &block) const +void TraceLogElement::appendToBlock(Block & block) const { 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(event_time); + columns[i++]->insert(ClickHouseRevision::get()); columns[i++]->insert(static_cast(timer_type)); + columns[i++]->insert(thread_number); columns[i++]->insertData(query_id.data(), query_id.size()); columns[i++]->insert(trace); diff --git a/dbms/src/Interpreters/TraceLog.h b/dbms/src/Interpreters/TraceLog.h index d5b38b69440..e86e789806f 100644 --- a/dbms/src/Interpreters/TraceLog.h +++ b/dbms/src/Interpreters/TraceLog.h @@ -15,7 +15,8 @@ struct TraceLogElement static const TimerDataType::Values timer_values; time_t event_time{}; - TimerType timer_type; + TimerType timer_type{}; + UInt32 thread_number{}; String query_id{}; Array trace{}; diff --git a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h index f1eaa3c9993..346c39b3344 100644 --- a/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h +++ b/dbms/src/Interpreters/TranslateQualifiedNamesVisitor.h @@ -24,7 +24,7 @@ public: struct Data { - NameSet source_columns; + const NameSet source_columns; const std::vector & tables; std::unordered_set join_using_columns; bool has_columns; diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 8cd017f0710..33c95cdf1aa 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -1337,7 +1337,6 @@ bool ParserWithOptionalAlias::parseImpl(Pos & pos, ASTPtr & node, Expected & exp if (auto * ast_with_alias = dynamic_cast(node.get())) { getIdentifierName(alias_node, ast_with_alias->alias); - ast_with_alias->prefer_alias_to_column_name = prefer_alias_to_column_name; } else { diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index dca82f72f12..9a87a78a5a3 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -274,13 +274,12 @@ protected: class ParserWithOptionalAlias : public IParserBase { public: - ParserWithOptionalAlias(ParserPtr && elem_parser_, bool allow_alias_without_as_keyword_, bool prefer_alias_to_column_name_ = false) - : 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_) {} + 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_) + {} protected: ParserPtr elem_parser; bool allow_alias_without_as_keyword; - bool prefer_alias_to_column_name; const char * getName() const { return "element of expression with optional alias"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index b948a22ce2a..4c8ef64b804 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -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(std::make_unique(), - 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) { return ParserList( - std::make_unique(allow_alias_without_as_keyword, prefer_alias_to_column_name), + std::make_unique(allow_alias_without_as_keyword), std::make_unique(TokenType::Comma)) .parse(pos, node, expected); } diff --git a/dbms/src/Parsers/ExpressionListParsers.h b/dbms/src/Parsers/ExpressionListParsers.h index 28a5bb5b2b9..f33965a9d05 100644 --- a/dbms/src/Parsers/ExpressionListParsers.h +++ b/dbms/src/Parsers/ExpressionListParsers.h @@ -322,7 +322,7 @@ using ParserExpression = ParserLambdaExpression; class ParserExpressionWithOptionalAlias : public IParserBase { public: - ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool prefer_alias_to_column_name_ = false); + ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword); protected: ParserPtr impl; @@ -339,12 +339,11 @@ protected: class ParserExpressionList : public IParserBase { public: - ParserExpressionList(bool allow_alias_without_as_keyword_, bool prefer_alias_to_column_name_ = false) - : allow_alias_without_as_keyword(allow_alias_without_as_keyword_), prefer_alias_to_column_name(prefer_alias_to_column_name_) {} + ParserExpressionList(bool allow_alias_without_as_keyword_) + : allow_alias_without_as_keyword(allow_alias_without_as_keyword_) {} protected: bool allow_alias_without_as_keyword; - bool prefer_alias_to_column_name; const char * getName() const { return "list of expressions"; } bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); @@ -354,8 +353,8 @@ protected: class ParserNotEmptyExpressionList : public IParserBase { public: - ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword, bool prefer_alias_to_column_name = false) - : nested_parser(allow_alias_without_as_keyword, prefer_alias_to_column_name) {} + ParserNotEmptyExpressionList(bool allow_alias_without_as_keyword) + : nested_parser(allow_alias_without_as_keyword) {} private: ParserExpressionList nested_parser; protected: diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 644d63788cb..afef7842ef6 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -44,7 +44,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_offset("OFFSET"); 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. ParserExpressionWithOptionalAlias exp_elem(false); ParserOrderByExpressionList order_list; diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index a2f3a47def3..1dcebc2959b 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -43,7 +43,7 @@ bool ProtobufRowInputFormat::readRow(MutableColumns & columns, RowReadExtension read_columns[column_index] = true; allow_add_row = false; } - } while (reader.maybeCanReadValue()); + } while (reader.canReadMoreValues()); } // Fill non-visited columns with the default values. @@ -62,7 +62,7 @@ bool ProtobufRowInputFormat::allowSyncAfterError() const void ProtobufRowInputFormat::syncAfterError() { - reader.endMessage(); + reader.endMessage(true); } diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp index 5b8d80cb062..19b496e0e60 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -28,7 +28,10 @@ KafkaBlockInputStream::~KafkaBlockInputStream() return; if (broken) + { buffer->subBufferAs()->unsubscribe(); + buffer->reset(); + } storage.pushBuffer(buffer); } diff --git a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp index 5c7a8222a69..db3de302dd8 100644 --- a/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp +++ b/dbms/src/Storages/Kafka/ReadBufferFromKafkaConsumer.cpp @@ -90,6 +90,11 @@ void ReadBufferFromKafkaConsumer::subscribe(const Names & topics) void ReadBufferFromKafkaConsumer::unsubscribe() { LOG_TRACE(log, "Re-joining claimed consumer after failure"); + + messages.clear(); + current = messages.begin(); + BufferBase::set(nullptr, 0, 0); + consumer->unsubscribe(); } diff --git a/dbms/src/Storages/MergeTree/KeyCondition.cpp b/dbms/src/Storages/MergeTree/KeyCondition.cpp index f08a9c35c9e..790c72329a2 100644 --- a/dbms/src/Storages/MergeTree/KeyCondition.cpp +++ b/dbms/src/Storages/MergeTree/KeyCondition.cpp @@ -689,6 +689,9 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo MonotonicFunctionsChain chain; std::string func_name = func->name; + if (atom_map.find(func_name) == std::end(atom_map)) + return false; + if (args.size() == 1) { 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; 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.monotonic_functions_chain = std::move(chain); diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index f5b656a9364..6aa9b07df73 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -27,7 +27,8 @@ void MergeTreeBlockOutputStream::write(const Block & block) 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'. - storage.background_task_handle->wake(); + if (storage.background_task_handle) + storage.background_task_handle->wake(); } } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 8b38f1ff32e..5ed783c034f 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -381,18 +381,18 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm } rows_count += rows; - { /// Creating block for update Block indices_update_block(skip_indexes_columns); + size_t skip_index_current_mark = 0; + /// Filling and writing skip indices like in IMergedBlockOutputStream::writeColumn for (size_t i = 0; i < storage.skip_indices.size(); ++i) { const auto index = storage.skip_indices[i]; auto & stream = *skip_indices_streams[i]; size_t prev_pos = 0; - - size_t skip_index_current_mark = 0; + skip_index_current_mark = skip_index_mark; while (prev_pos < rows) { UInt64 limit = 0; @@ -417,6 +417,8 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm /// to be compatible with normal .mrk2 file format if (storage.canUseAdaptiveGranularity()) writeIntBinary(1UL, stream.marks); + + ++skip_index_current_mark; } } @@ -435,9 +437,9 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm } } prev_pos = pos; - ++skip_index_current_mark; } } + skip_index_mark = skip_index_current_mark; } { diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 467660413b3..3acb01c3c0a 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -68,6 +68,7 @@ private: String part_path; size_t rows_count = 0; + size_t skip_index_mark = 0; std::unique_ptr index_file_stream; std::unique_ptr index_stream; diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 4c029fab677..3487a1becf5 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -23,8 +25,6 @@ #include #include #include -#include -#include #include #include #include diff --git a/dbms/tests/instructions/sanitizers.md b/dbms/tests/instructions/sanitizers.md index c21ff9b0a9b..3ec8d30ae09 100644 --- a/dbms/tests/instructions/sanitizers.md +++ b/dbms/tests/instructions/sanitizers.md @@ -67,5 +67,5 @@ sudo -u clickhouse UBSAN_OPTIONS='print_stacktrace=1' ./clickhouse-ubsan server # 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 .. ``` diff --git a/dbms/tests/integration/test_config_corresponding_root/__init__.py b/dbms/tests/integration/test_config_corresponding_root/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_config_corresponding_root/configs/config.d/bad.xml b/dbms/tests/integration/test_config_corresponding_root/configs/config.d/bad.xml new file mode 100644 index 00000000000..9398be257eb --- /dev/null +++ b/dbms/tests/integration/test_config_corresponding_root/configs/config.d/bad.xml @@ -0,0 +1,4 @@ + + + + diff --git a/dbms/tests/integration/test_config_corresponding_root/configs/config.xml b/dbms/tests/integration/test_config_corresponding_root/configs/config.xml new file mode 100644 index 00000000000..154ebf6c35e --- /dev/null +++ b/dbms/tests/integration/test_config_corresponding_root/configs/config.xml @@ -0,0 +1,415 @@ + + + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + + 8123 + 9000 + + + + + + + + + /etc/clickhouse-server/server.crt + /etc/clickhouse-server/server.key + + /etc/clickhouse-server/dhparam.pem + none + true + true + sslv2,sslv3 + true + + + + true + true + sslv2,sslv3 + true + + + + RejectCertificateHandler + + + + + + + + + 9009 + + + + + + + + + + + + + + + + + + + + 4096 + 3 + + + 100 + + + + + + 8589934592 + + + 5368709120 + + + + /var/lib/clickhouse/ + + + /var/lib/clickhouse/tmp/ + + + /var/lib/clickhouse/user_files/ + + + users.xml + + + default + + + + + + default + + + + + + + + + false + + + + + + + + localhost + 9000 + + + + + + + localhost + 9000 + + + + + localhost + 9000 + + + + + + + localhost + 9440 + 1 + + + + + + + localhost + 9000 + + + + + localhost + 1 + + + + + + + + + + + + + + + + + 3600 + + + + 3600 + + + 60 + + + + + + + + + + system + query_log
+ + toYYYYMM(event_date) + + 7500 +
+ + + + system + query_thread_log
+ toYYYYMM(event_date) + 7500 +
+ + + + + + + + + + + + + + + *_dictionary.xml + + + + + + + + + + /clickhouse/task_queue/ddl + + + + + + + + + + + + + + + + click_cost + any + + 0 + 3600 + + + 86400 + 60 + + + + max + + 0 + 60 + + + 3600 + 300 + + + 86400 + 3600 + + + + + + /var/lib/clickhouse/format_schemas/ + + + +
diff --git a/dbms/tests/integration/test_config_corresponding_root/configs/users.xml b/dbms/tests/integration/test_config_corresponding_root/configs/users.xml new file mode 100644 index 00000000000..24b8f628c3a --- /dev/null +++ b/dbms/tests/integration/test_config_corresponding_root/configs/users.xml @@ -0,0 +1,130 @@ + + + + + + + + 10000000000 + + + 0 + + + random + + + + + 1 + + + + + + + + + + + + + ::/0 + + + + default + + + default + + + + + + + a = 1 + + + + + a + b < 1 or c - d > 5 + + + + + c = 1 + + + + + + + + + + + + + + + + + 3600 + + + 0 + 0 + 0 + 0 + 0 + + + + diff --git a/dbms/tests/integration/test_config_corresponding_root/test.py b/dbms/tests/integration/test_config_corresponding_root/test.py new file mode 100644 index 00000000000..fd5d3eae3ff --- /dev/null +++ b/dbms/tests/integration/test_config_corresponding_root/test.py @@ -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 diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cmd.xml b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cmd.xml index a12428af5c7..9f1e259e2d7 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cmd.xml +++ b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cmd.xml @@ -1,5 +1,5 @@ - + cmd @@ -16,4 +16,4 @@ - + diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_x.xml b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_x.xml index 6eed3fbc891..097fc7cf503 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_x.xml +++ b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_x.xml @@ -1,30 +1,29 @@ - - - - dep_x - - - localhost - 9000 - default - - dict - dep_z
-
- - 5 - - - - - - id - - - a - String - XX - - -
-
+ + + dep_x + + + localhost + 9000 + default + + dict + dep_z
+
+ + 5 + + + + + + id + + + a + String + XX + + +
+
diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_y.xml b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_y.xml index 7891e945566..8806c724111 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_y.xml +++ b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_y.xml @@ -1,40 +1,39 @@ - - + - dep_y - - - localhost - 9000 - default - - test - small_dict_source
-
- - 5 - - - - - - id - - - b - Int32 - -1 - - - c - Float64 - -2 - - - a - String - YY - - -
-
+ dep_y + + + localhost + 9000 + default + + test + small_dict_source
+
+ + 5 + + + + + + id + + + b + Int32 + -1 + + + c + Float64 + -2 + + + a + String + YY + + + + diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_z.xml b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_z.xml index e17107d2ac2..3fec7ac3cff 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_z.xml +++ b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_z.xml @@ -1,4 +1,4 @@ - + dep_z @@ -34,3 +34,4 @@ + diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.xml b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.xml index 14b82305b4d..0e6db1f1637 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.xml +++ b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.xml @@ -1,36 +1,53 @@ - - - file - - - /etc/clickhouse-server/config.d/dictionary_preset_file.txt - TabSeparated - - - 1 - - key - aInt32 - 0 - - - - - - no_file - - - /etc/clickhouse-server/config.d/dictionary_preset_no_file.txt - TabSeparated - - - 1 - - key - aInt32 - 0 - - - - + + + file + + + /etc/clickhouse-server/config.d/dictionary_preset_file.txt + TabSeparated + + + 1 + + key + aInt32 + 0 + + + + + + no_file + + + /etc/clickhouse-server/config.d/dictionary_preset_no_file.txt + TabSeparated + + + 1 + + key + aInt32 + 0 + + + + + + no_file_2 + + + /etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt + TabSeparated + + + 1 + + key + aInt32 + 0 + + + + diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_longload.xml b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_longload.xml index b96a6151eb9..f5d4cdec583 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_longload.xml +++ b/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_longload.xml @@ -1,5 +1,5 @@ - + longload @@ -16,4 +16,4 @@ - + diff --git a/dbms/tests/integration/test_dictionaries/generate_dictionaries.py b/dbms/tests/integration/test_dictionaries/generate_dictionaries.py index d0b0dd67a3a..c644bd8f644 100644 --- a/dbms/tests/integration/test_dictionaries/generate_dictionaries.py +++ b/dbms/tests/integration/test_dictionaries/generate_dictionaries.py @@ -52,32 +52,32 @@ def generate_structure(): def generate_dictionaries(path, structure): dictionary_skeleton = ''' - - - {name} + + + {name} - - {source} - + + {source} + - - 0 - 0 - + + 0 + 0 + - - {layout} - + + {layout} + - - {key} + + {key} - %s + %s - {parent} - - - ''' + {parent} + + + ''' attribute_skeleton = ''' %s_ diff --git a/dbms/tests/integration/test_dictionaries/test.py b/dbms/tests/integration/test_dictionaries/test.py index dfb27cd2ed7..251fe7f31ee 100644 --- a/dbms/tests/integration/test_dictionaries/test.py +++ b/dbms/tests/integration/test_dictionaries/test.py @@ -294,7 +294,7 @@ def test_reload_after_loading(started_cluster): 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 # 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("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n" 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" diff --git a/dbms/tests/integration/test_external_dictionaries/dictionary.py b/dbms/tests/integration/test_external_dictionaries/dictionary.py index bdddc7a9604..e84eda9bea7 100644 --- a/dbms/tests/integration/test_external_dictionaries/dictionary.py +++ b/dbms/tests/integration/test_external_dictionaries/dictionary.py @@ -290,19 +290,19 @@ class Dictionary(object): def generate_config(self): with open(self.config_path, 'w') as result: result.write(''' - - - - 3 - 5 - - {name} - {structure} - - {source} - - - + + + + 3 + 5 + + {name} + {structure} + + {source} + + + '''.format( name=self.name, structure=self.structure.get_structure_str(), diff --git a/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile b/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile index 85e256dfed6..76125702076 100644 --- a/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile +++ b/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/Dockerfile @@ -3,5 +3,6 @@ FROM php:7.3-cli COPY ./client.crt client.crt COPY ./client.key client.key COPY ./test.php test.php +COPY ./test_ssl.php test_ssl.php RUN docker-php-ext-install pdo pdo_mysql diff --git a/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php b/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php index 86354381835..dc5b55ad1d5 100644 --- a/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php +++ b/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test.php @@ -12,9 +12,6 @@ $options = [ 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); diff --git a/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php b/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php new file mode 100644 index 00000000000..86354381835 --- /dev/null +++ b/dbms/tests/integration/test_mysql_protocol/clients/php-mysqlnd/test_ssl.php @@ -0,0 +1,27 @@ + 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"; +} +?> diff --git a/dbms/tests/integration/test_mysql_protocol/test.py b/dbms/tests/integration/test_mysql_protocol/test.py index f0c696b4dc7..4a8d6f0b608 100644 --- a/dbms/tests/integration/test_mysql_protocol/test.py +++ b/dbms/tests/integration/test_mysql_protocol/test.py @@ -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) assert code == 0 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' diff --git a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml index 1c293f66761..19eed6ebd6a 100644 --- a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml +++ b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/postgres_odbc_hashed_dictionary.xml @@ -1,38 +1,38 @@ - - - postgres_odbc_hashed - - - clickhouse.test_table
- DSN=postgresql_odbc; - postgres -
- - - 5 - 5 - - - - + + + postgres_odbc_hashed + + + clickhouse.test_table
+ DSN=postgresql_odbc; + postgres +
+ + + 5 + 5 + + + + - - - column1 - + + + column1 + - - column1 - Int64 - 1 - + + column1 + Int64 + 1 + - - column2 - String - '' - + + column2 + String + '' + - -
-
+ + + diff --git a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_cached_dictionary.xml b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_cached_dictionary.xml index f7eaea03a5e..45f3966ee8a 100644 --- a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_cached_dictionary.xml +++ b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_cached_dictionary.xml @@ -1,46 +1,46 @@ - - - sqlite3_odbc_cached - - - t3
- DSN=sqlite3_odbc -
- + + + sqlite3_odbc_cached + + + t3
+ DSN=sqlite3_odbc +
+ - - 128 - + + 128 + - - 1 - 1 - + + 1 + 1 + - - - X - + + + X + - - X - Int64 - 1 - + + X + Int64 + 1 + - - Y - Int64 - 1 - + + Y + Int64 + 1 + - - Z - UInt8 - 1 - + + Z + UInt8 + 1 + - -
-
+ + + diff --git a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml index 76ba4ad27f3..18a14b896bd 100644 --- a/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml +++ b/dbms/tests/integration/test_odbc_interaction/configs/dictionaries/sqlite3_odbc_hashed_dictionary.xml @@ -1,46 +1,46 @@ - - - sqlite3_odbc_hashed - - - t2
- DSN=sqlite3_odbc - SELECT Z from t2 where X=1 -
- + + + sqlite3_odbc_hashed + + + t2
+ DSN=sqlite3_odbc + SELECT Z from t2 where X=1 +
+ - - 1 - 1 - + + 1 + 1 + - - - + + + - - - X - + + + X + - - X - Int64 - 1 - + + X + Int64 + 1 + - - Y - Int64 - 1 - + + Y + Int64 + 1 + - - Z - UInt8 - 1 - + + Z + UInt8 + 1 + - -
-
+ + + diff --git a/dbms/tests/integration/test_storage_kafka/test.py b/dbms/tests/integration/test_storage_kafka/test.py index 3f38b068a22..0324307a7a1 100644 --- a/dbms/tests/integration/test_storage_kafka/test.py +++ b/dbms/tests/integration/test_storage_kafka/test.py @@ -140,6 +140,7 @@ def test_kafka_settings_old_syntax(kafka_cluster): result += instance.query('SELECT * FROM test.kafka') if kafka_check_result(result): break + time.sleep(0.5) kafka_check_result(result, True) @@ -170,10 +171,11 @@ def test_kafka_settings_new_syntax(kafka_cluster): kafka_produce('new', messages) result = '' - while True: + for i in range(50): result += instance.query('SELECT * FROM test.kafka') if kafka_check_result(result): break + time.sleep(0.5) kafka_check_result(result, True) @@ -194,10 +196,11 @@ def test_kafka_csv_with_delimiter(kafka_cluster): kafka_produce('csv', messages) result = '' - while True: + for i in range(50): result += instance.query('SELECT * FROM test.kafka') if kafka_check_result(result): break + time.sleep(0.5) kafka_check_result(result, True) @@ -218,10 +221,11 @@ def test_kafka_tsv_with_delimiter(kafka_cluster): kafka_produce('tsv', messages) result = '' - while True: + for i in range(50): result += instance.query('SELECT * FROM test.kafka') if kafka_check_result(result): break + time.sleep(0.5) kafka_check_result(result, True) @@ -246,10 +250,11 @@ def test_kafka_json_without_delimiter(kafka_cluster): kafka_produce('json', [messages]) result = '' - while True: + for i in range(50): result += instance.query('SELECT * FROM test.kafka') if kafka_check_result(result): break + time.sleep(0.5) kafka_check_result(result, True) @@ -269,10 +274,11 @@ def test_kafka_protobuf(kafka_cluster): kafka_produce_protobuf_messages('pb', 21, 29) result = '' - while True: + for i in range(50): result += instance.query('SELECT * FROM test.kafka') if kafka_check_result(result): break + time.sleep(0.5) kafka_check_result(result, True) @@ -299,11 +305,11 @@ def test_kafka_materialized_view(kafka_cluster): messages.append(json.dumps({'key': i, 'value': i})) kafka_produce('mv', messages) - while True: - time.sleep(1) + for i in range(50): result = instance.query('SELECT * FROM test.view') if kafka_check_result(result): break + time.sleep(0.5) kafka_check_result(result, True) instance.query(''' @@ -348,11 +354,11 @@ def test_kafka_flush_on_big_message(kafka_cluster): except kafka.errors.GroupCoordinatorNotAvailableError: continue - while True: - time.sleep(1) + for i in range(50): result = instance.query('SELECT count() FROM test.view') if int(result) == kafka_messages*batch_messages: break + time.sleep(0.5) 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]) result = '' - while True: - time.sleep(1) + for i in range(50): result += instance.query('SELECT _key, key, _topic, value, _offset FROM test.kafka') if kafka_check_result(result, False, 'test_kafka_virtual1.reference'): break + time.sleep(0.5) 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})) kafka_produce('virt2', messages) - while True: - time.sleep(1) + for i in range(50): result = instance.query('SELECT kafka_key, key, topic, value, offset FROM test.view') if kafka_check_result(result, False, 'test_kafka_virtual2.reference'): break + time.sleep(0.5) kafka_check_result(result, True, 'test_kafka_virtual2.reference') instance.query(''' diff --git a/dbms/tests/queries/0_stateless/00800_low_cardinality_join.sql b/dbms/tests/queries/0_stateless/00800_low_cardinality_join.sql index 07ad6d54624..9734b1ef31c 100644 --- a/dbms/tests/queries/0_stateless/00800_low_cardinality_join.sql +++ b/dbms/tests/queries/0_stateless/00800_low_cardinality_join.sql @@ -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(toNullable(dummy)) as val from system.one) any left join (select toLowCardinality(toNullable(dummy)) as val from system.one) using val; 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 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 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 toLowCardinality(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 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 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 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 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(toNullable(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 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 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 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 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 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 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 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 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 * 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; diff --git a/dbms/tests/queries/0_stateless/00808_not_optimize_predicate.sql b/dbms/tests/queries/0_stateless/00808_not_optimize_predicate.sql index 21424152da1..542bea690e8 100644 --- a/dbms/tests/queries/0_stateless/00808_not_optimize_predicate.sql +++ b/dbms/tests/queries/0_stateless/00808_not_optimize_predicate.sql @@ -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; 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 id FROM test_00808 GROUP BY id LIMIT 1 BY id) WHERE id = 1; -- { serverError 277 } diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format.proto b/dbms/tests/queries/0_stateless/00825_protobuf_format.proto index b588619f488..0d9bdd83ccd 100644 --- a/dbms/tests/queries/0_stateless/00825_protobuf_format.proto +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format.proto @@ -143,3 +143,9 @@ message StrPerson { MeasureUnits measureUnits = 21; NestinessA nestiness_a = 22; }; + +message NumberAndSquare +{ + uint32 number = 1; + uint64 square = 2; +}; diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_input.insh b/dbms/tests/queries/0_stateless/00825_protobuf_format_input.insh index d615f6e4e6d..39a2f17c98f 100644 --- a/dbms/tests/queries/0_stateless/00825_protobuf_format_input.insh +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_input.insh @@ -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 '\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 '\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 '\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 '\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 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 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 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'" diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_input.reference b/dbms/tests/queries/0_stateless/00825_protobuf_format_input.reference index e374ff0bcf6..884cc74c4e5 100644 --- a/dbms/tests/queries/0_stateless/00825_protobuf_format_input.reference +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_input.reference @@ -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] 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 [] +0 0 +2 4 +3 9 diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_input.sh b/dbms/tests/queries/0_stateless/00825_protobuf_format_input.sh index d56864e0b0e..d28b70bb002 100755 --- a/dbms/tests/queries/0_stateless/00825_protobuf_format_input.sh +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_input.sh @@ -7,34 +7,37 @@ set -e -o pipefail # Run the client. $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, - name String, - surname String, - gender Enum8('male'=1, 'female'=0), - birthDate Date, - photo Nullable(String), - phoneNumber Nullable(FixedString(13)), - isOnline UInt8, - visitTime Nullable(DateTime), - age UInt8, - zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823, - 'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120, - 'pisces'=219), - songs Array(String), - color Array(UInt8), - hometown LowCardinality(String), - location Array(Decimal32(6)), - pi Nullable(Float64), - lotteryWin Nullable(Decimal64(2)), - someRatio Float32, - temperature Decimal32(1), - randomBigNumber Int64, - measureUnits Nested (unit String, coef Float32), - nestiness_a_b_c_d Nullable(UInt32), - `nestiness_a_B.c_E` Array(UInt32) - ) ENGINE = MergeTree ORDER BY tuple(); +CREATE TABLE in_persons_00825 (uuid UUID, + name String, + surname String, + gender Enum8('male'=1, 'female'=0), + birthDate Date, + photo Nullable(String), + phoneNumber Nullable(FixedString(13)), + isOnline UInt8, + visitTime Nullable(DateTime), + age UInt8, + zodiacSign Enum16('aries'=321, 'taurus'=420, 'gemini'=521, 'cancer'=621, 'leo'=723, 'virgo'=823, + 'libra'=923, 'scorpius'=1023, 'sagittarius'=1122, 'capricorn'=1222, 'aquarius'=120, + 'pisces'=219), + songs Array(String), + color Array(UInt8), + hometown LowCardinality(String), + location Array(Decimal32(6)), + pi Nullable(Float64), + lotteryWin Nullable(Decimal64(2)), + someRatio Float32, + temperature Decimal32(1), + randomBigNumber Int64, + measureUnits Nested (unit String, coef Float32), + nestiness_a_b_c_d Nullable(UInt32), + `nestiness_a_B.c_E` Array(UInt32) + ) ENGINE = MergeTree ORDER BY tuple(); + +CREATE TABLE in_squares_00825 (number UInt32, square UInt32) ENGINE = MergeTree ORDER BY tuple(); EOF # To generate the file 00825_protobuf_format_input.insh use the following commands: @@ -42,6 +45,8 @@ EOF # build/utils/test-data-generator/ProtobufDelimitedMessagesSerializer 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;" diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference index 7387762c8ec..9d20d778ff6 100644 Binary files a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference and b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.reference differ diff --git a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh index 3fd2a5abd18..403e128e7a1 100755 --- a/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh +++ b/dbms/tests/queries/0_stateless/00825_protobuf_format_output.sh @@ -11,46 +11,53 @@ set -e -o pipefail # Run the client. $CLICKHOUSE_CLIENT --multiquery <'; -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 * 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 * 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 diff --git a/dbms/tests/queries/0_stateless/00971_query_id_in_logs.sh b/dbms/tests/queries/0_stateless/00971_query_id_in_logs.sh index a4ef7671f48..83c72be265c 100755 --- a/dbms/tests/queries/0_stateless/00971_query_id_in_logs.sh +++ b/dbms/tests/queries/0_stateless/00971_query_id_in_logs.sh @@ -1,9 +1,11 @@ #!/usr/bin/env bash +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=trace + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh set -e # 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+>' ||: diff --git a/dbms/tests/queries/0_stateless/00974_adaptive_granularity_secondary_index.reference b/dbms/tests/queries/0_stateless/00974_adaptive_granularity_secondary_index.reference new file mode 100644 index 00000000000..5878ba47225 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_adaptive_granularity_secondary_index.reference @@ -0,0 +1,2 @@ +1000 +1000 diff --git a/dbms/tests/queries/0_stateless/00974_adaptive_granularity_secondary_index.sql b/dbms/tests/queries/0_stateless/00974_adaptive_granularity_secondary_index.sql new file mode 100644 index 00000000000..328ec86f060 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_adaptive_granularity_secondary_index.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00974_bitmapContains_with_primary_key.reference b/dbms/tests/queries/0_stateless/00974_bitmapContains_with_primary_key.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_bitmapContains_with_primary_key.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00974_bitmapContains_with_primary_key.sql b/dbms/tests/queries/0_stateless/00974_bitmapContains_with_primary_key.sql new file mode 100644 index 00000000000..81dd7cab9f4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_bitmapContains_with_primary_key.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00974_final_predicate_push_down.reference b/dbms/tests/queries/0_stateless/00974_final_predicate_push_down.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_final_predicate_push_down.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/0_stateless/00974_final_predicate_push_down.sql b/dbms/tests/queries/0_stateless/00974_final_predicate_push_down.sql new file mode 100644 index 00000000000..96bcbf9aae6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_final_predicate_push_down.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00974_fix_join_on.reference b/dbms/tests/queries/0_stateless/00974_fix_join_on.reference new file mode 100644 index 00000000000..90c76d8b931 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_fix_join_on.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00974_fix_join_on.sql b/dbms/tests/queries/0_stateless/00974_fix_join_on.sql new file mode 100644 index 00000000000..feffe046d4f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_fix_join_on.sql @@ -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; diff --git a/dbms/tests/queries/0_stateless/00974_full_outer_join.reference b/dbms/tests/queries/0_stateless/00974_full_outer_join.reference new file mode 100644 index 00000000000..82c1bd051d3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_full_outer_join.reference @@ -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 diff --git a/dbms/tests/queries/0_stateless/00974_full_outer_join.sql b/dbms/tests/queries/0_stateless/00974_full_outer_join.sql new file mode 100644 index 00000000000..fda9d70e444 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_full_outer_join.sql @@ -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 diff --git a/dbms/tests/queries/0_stateless/00974_query_profiler.reference b/dbms/tests/queries/0_stateless/00974_query_profiler.reference new file mode 100644 index 00000000000..e37cf5f7642 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_query_profiler.reference @@ -0,0 +1,4 @@ +0 0 +1 +1000000000 0 +1 diff --git a/dbms/tests/queries/0_stateless/00974_query_profiler.sql b/dbms/tests/queries/0_stateless/00974_query_profiler.sql new file mode 100644 index 00000000000..b3d70bc6ac3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00974_query_profiler.sql @@ -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%'; diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 63648d95b77..1c97cb134e2 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -32,7 +32,7 @@ Possible values: - 0 — Disabled. - 1 — Enabled. -Default value: 0. +Default value: 1. **Usage** diff --git a/docs/en/query_language/dicts/external_dicts_dict_structure.md b/docs/en/query_language/dicts/external_dicts_dict_structure.md index d8ecf4e2efa..d5377c39289 100644 --- a/docs/en/query_language/dicts/external_dicts_dict_structure.md +++ b/docs/en/query_language/dicts/external_dicts_dict_structure.md @@ -42,7 +42,7 @@ A structure can contain either `` or `` . ### Numeric Key -Format: `UInt64`. +Type: `UInt64`. Configuration example: @@ -54,7 +54,7 @@ Configuration example: Configuration fields: -- name – The name of the column with keys. +- `name` – The name of the column with keys. ### Composite Key @@ -93,7 +93,7 @@ Configuration example: ... Name - Type + ClickHouseDataType rand64() true @@ -108,11 +108,11 @@ Configuration fields: Tag | Description | Required ----|-------------|--------- `name`| Column name. | Yes -`type`| Column type.
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 -`null_value` | Default value for a non-existing element.
In the example, it is an empty string. | Yes -`expression` | [Expression](../syntax.md#syntax-expressions) that ClickHouse executes on the value.
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.

Default value: no expression. | No +`type`| ClickHouse data type.
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.
[Nullable](../../data_types/nullable.md) is not supported. | Yes +`null_value` | Default value for a non-existing element.
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.
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.

Default value: no expression. | No `hierarchical` | Hierarchical support. Mirrored to the parent identifier.

Default value: `false`. | No -`injective` | Flag that shows whether the `id -> attribute` image is injective.
If `true`, then you can optimize the `GROUP BY` clause.

Default value: `false`. | No +`injective` | Flag that shows whether the `id -> attribute` image is [injective](https://en.wikipedia.org/wiki/Injective_function).
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.

Default value: `false`. | No `is_object_id` | Flag that shows whether the query is executed for a MongoDB document by `ObjectID`.

Default value: `false`. | No [Original article](https://clickhouse.yandex/docs/en/query_language/dicts/external_dicts_dict_structure/) diff --git a/docs/en/query_language/functions/hash_functions.md b/docs/en/query_language/functions/hash_functions.md index 368cfa1622b..296cca1e712 100644 --- a/docs/en/query_language/functions/hash_functions.md +++ b/docs/en/query_language/functions/hash_functions.md @@ -1,16 +1,16 @@ # 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} -[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, ...) ``` -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. **Parameters** @@ -19,7 +19,7 @@ The function takes a variable number of input parameters. Parameters can be any **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** @@ -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} -Produces 64-bit [SipHash](https://131002.net/siphash/) hash value. +Produces a 64-bit [SipHash](https://131002.net/siphash/) hash value. ``` 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. +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** The function takes a variable number of input parameters. Parameters can be any of the [supported data types](../../data_types/index.md). **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** @@ -77,13 +82,13 @@ Differs from sipHash64 in that the final xor-folding state is only done up to 12 ## cityHash64 -Produces 64-bit hash value. +Produces a 64-bit [CityHash](https://github.com/google/cityhash) hash value. ``` 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** @@ -91,7 +96,7 @@ The function takes a variable number of input parameters. Parameters can be any **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** @@ -100,7 +105,7 @@ Call example: ```sql SELECT cityHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS CityHash, toTypeName(CityHash) AS type ``` -``` +```text ┌─────────────CityHash─┬─type───┐ │ 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: -``` -SELECT sum(cityHash64(*)) FROM table +```sql +SELECT groupBitXor(cityHash64(*)) FROM table ``` @@ -157,7 +162,7 @@ The function takes a variable number of input parameters. Parameters can be any **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** @@ -172,15 +177,15 @@ SELECT farmHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:0 ## 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. -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 Calculates HiveHash from a string. 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 @@ -196,7 +201,7 @@ The function takes a variable number of input parameters. Parameters can be any **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** @@ -259,8 +264,8 @@ Both functions take a variable number of input parameters. Parameters can be any **Returned Value** -- The `murmurHash3_32` function returns hash value having the [UInt32](../../data_types/int_uint.md) data type. -- The `murmurHash3_64` function returns hash value having the [UInt64](../../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 a [UInt64](../../data_types/int_uint.md) data type hash value. **Example** @@ -283,11 +288,11 @@ murmurHash3_128( expr ) **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** -Hash value having [FixedString(16) data type](../../data_types/fixedstring.md). +A [FixedString(16)](../../data_types/fixedstring.md) data type hash value. **Example** diff --git a/docs/en/query_language/misc.md b/docs/en/query_language/misc.md index c3e7e598547..31bfea5dc4d 100644 --- a/docs/en/query_language/misc.md +++ b/docs/en/query_language/misc.md @@ -39,9 +39,11 @@ The `CHECK TABLE` query supports the following table engines: - [StripeLog](../operations/table_engines/stripelog.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** diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e5e4bad1fa6..86c16e92da3 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -31,7 +31,7 @@ ClickHouse применяет настройку в тех случаях, ко - 0 — выключена. - 1 — включена. -Значение по умолчанию: 0. +Значение по умолчанию: 1. **Использование** diff --git a/docs/ru/query_language/dicts/external_dicts_dict_structure.md b/docs/ru/query_language/dicts/external_dicts_dict_structure.md index 482f47d4a8f..b553a6fdbea 100644 --- a/docs/ru/query_language/dicts/external_dicts_dict_structure.md +++ b/docs/ru/query_language/dicts/external_dicts_dict_structure.md @@ -1,4 +1,3 @@ - # Ключ и поля словаря Секция `` описывает ключ словаря и поля, доступные для запросов. @@ -24,25 +23,24 @@ В структуре описываются столбцы: -- `` - [ключевой столбец](external_dicts_dict_structure.md). -- `` - [столбец данных](external_dicts_dict_structure.md). Столбцов может быть много. +- `` — [ключевой столбец](external_dicts_dict_structure.md#ext_dict_structure-key). +- `` — [столбец данных](external_dicts_dict_structure.md#ext_dict_structure-attributes). Столбцов может быть много. - -## Ключ +## Ключ {#ext_dict_structure-key} ClickHouse поддерживает следующие виды ключей: -- Числовой ключ. Формат UInt64. Описывается в теге ``. -- Составной ключ. Набор значений разного типа. Описывается в теге ``. +- Числовой ключ. UInt64. Описывается в теге ``. +- Составной ключ. Набор значений разного типа. Описывается в теге ``. Структура может содержать либо `` либо ``. -!!! 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 "Совет" Cоставной ключ может состоять из одного элемента. Это даёт возможность использовать в качестве ключа, например, строку. @@ -93,7 +91,7 @@ ClickHouse поддерживает следующие виды ключей: ... Name - Type + ClickHouseDataType rand64() true @@ -105,13 +103,14 @@ ClickHouse поддерживает следующие виды ключей: Поля конфигурации: -- `name` - Имя столбца. -- `type` - Тип столбца. Задает способ интерпретации данных в источнике. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. -- `null_value` - Значение по умолчанию для несуществующего элемента. В примере - пустая строка. -- `expression` - Атрибут может быть выражением. Тег не обязательный. -- `hierarchical` - Поддержка иерархии. Отображение в идентификатор родителя. По умолчанию, `false`. -- `injective` - Признак инъективности отображения `id -> attribute`. Если `true`, то можно оптимизировать `GROUP BY`. По умолчанию, `false`. -- `is_object_id` - Признак того, что запрос выполняется к документу MongoDB по `ObjectID`. - +| Тег | Описание | Обязательный | +| ---- | ------------- | --------- | +| `name` | Имя столбца. | Да | +| `type` | Тип данных ClickHouse.
ClickHouse пытается привести значение из словаря к заданному типу данных. Например, в случае MySQL, в таблице-источнике поле может быть `TEXT`, `VARCHAR`, `BLOB`, но загружено может быть как `String`. [Nullable](../../data_types/nullable.md) не поддерживается. | Да | +| `null_value` | Значение по умолчанию для несуществующего элемента.
В примере это пустая строка. Нельзя указать значение `NULL`. | Да | +| `expression` | [Выражение](../syntax.md#syntax-expressions), которое ClickHouse выполняет со значением.
Выражением может быть имя столбца в удаленной SQL базе. Таким образом, вы можете использовать его для создания псевдонима удаленного столбца.

Значение по умолчанию: нет выражения. | Нет | +| `hierarchical` | Поддержка иерархии. Отображение в идентификатор родителя.

Значение по умолчанию: `false`. | Нет | +| `injective` | Признак [инъективности](https://ru.wikipedia.org/wiki/Инъекция_(математика)) отображения `id -> attribute`.
Если `true`, то обращения к словарям с включенной инъективностью могут быть автоматически переставлены ClickHouse за стадию `GROUP BY`, что как правило существенно сокращает их количество.

Значение по умолчанию: `false`. | Нет | +| `is_object_id` | Признак того, что запрос выполняется к документу MongoDB по `ObjectID`.

Значение по умолчанию: `false`. | Нет | [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/dicts/external_dicts_dict_structure/) diff --git a/docs/ru/query_language/functions/hash_functions.md b/docs/ru/query_language/functions/hash_functions.md index 62699d38d5b..e171b2bfa38 100644 --- a/docs/ru/query_language/functions/hash_functions.md +++ b/docs/ru/query_language/functions/hash_functions.md @@ -2,41 +2,130 @@ Функции хэширования могут использоваться для детерминированного псевдослучайного разбрасывания элементов. -## halfMD5 -Вычисляет MD5 от строки. Затем берёт первые 8 байт от хэша и интерпретирует их как UInt64 в big endian. -Принимает аргумент типа String. Возвращает UInt64. -Функция работает достаточно медленно (5 миллионов коротких строк в секунду на одном процессорном ядре). -Если вам не нужен конкретно MD5, то используйте вместо этого функцию sipHash64. +## halfMD5 {#hash_functions-halfmd5} -## MD5 +[Интерпретирует](../../query_language/functions/type_conversion_functions.md#type_conversion_functions-reinterpretAsString) все входные параметры как строки и вычисляет хэш [MD5](https://ru.wikipedia.org/wiki/MD5) для каждой из них. Затем объединяет хэши, берет первые 8 байт хэша результирующей строки и интерпретирует их как значение типа `UInt64` с big-endian порядком байтов. + +``` +halfMD5(par1, ...) +``` + +Функция относительно медленная (5 миллионов коротких строк в секунду на ядро процессора). +По возможности, используйте функцию [sipHash64](#hash_functions-siphash64) вместо неё. + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть любого [поддерживаемого типа данных](../../data_types/index.md). + +**Возвращаемое значение** + +Значение хэша с типом данных [UInt64](../../data_types/int_uint.md). + +**Пример** + +```sql +SELECT halfMD5(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS halfMD5hash, toTypeName(halfMD5hash) AS type +``` + +```text +┌────────halfMD5hash─┬─type───┐ +│ 186182704141653334 │ UInt64 │ +└────────────────────┴────────┘ +``` + +## MD5 {#hash_functions-md5} Вычисляет MD5 от строки и возвращает полученный набор байт в виде FixedString(16). Если вам не нужен конкретно MD5, а нужен неплохой криптографический 128-битный хэш, то используйте вместо этого функцию sipHash128. Если вы хотите получить такой же результат, как выдаёт утилита md5sum, напишите lower(hex(MD5(s))). -## sipHash64 -Вычисляет SipHash от строки. -Принимает аргумент типа String. Возвращает UInt64. -SipHash - криптографическая хэш-функция. Работает быстрее чем MD5 не менее чем в 3 раза. -Подробнее смотрите по ссылке: +## sipHash64 {#hash_functions-siphash64} + +Генерирует 64-х битное значение [SipHash](https://131002.net/siphash/). + +``` +sipHash64(par1,...) +``` + +Это криптографическая хэш-функция. Она работает по крайней мере в три раза быстрее, чем функция [MD5](#hash_functions-md5). + +Функция [интерпретирует](../../query_language/functions/type_conversion_functions.md#type_conversion_functions-reinterpretAsString) все входные параметры как строки и вычисляет хэш MD5 для каждой из них. Затем комбинирует хэши по следующему алгоритму. + +1. После хэширования всех входных параметров функция получает массив хэшей. +2. Функция принимает первый и второй элементы и вычисляет хэш для массива из них. +3. Затем функция принимает хэш-значение, вычисленное на предыдущем шаге, и третий элемент исходного хэш-массива, и вычисляет хэш для массива из них. +4. Предыдущий шаг повторяется для всех остальных элементов исходного хэш-массива. + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть любого [поддерживаемого типа данных](../../data_types/index.md). + +**Возвращаемое значение** + +Значение хэша с типом данных [UInt64](../../data_types/int_uint.md). + +**Пример** + +```sql +SELECT sipHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS SipHash, toTypeName(SipHash) AS type +``` + +``` +┌──────────────SipHash─┬─type───┐ +│ 13726873534472839665 │ UInt64 │ +└──────────────────────┴────────┘ +``` ## sipHash128 + Вычисляет SipHash от строки. Принимает аргумент типа String. Возвращает FixedString(16). Отличается от sipHash64 тем, что финальный xor-folding состояния делается только до 128 бит. ## cityHash64 -Вычисляет CityHash64 от строки или похожую хэш-функцию для произвольного количества аргументов произвольного типа. -Если аргумент имеет тип String, то используется CityHash. Это быстрая некриптографическая хэш-функция неплохого качества для строк. -Если аргумент имеет другой тип, то используется implementation specific быстрая некриптографическая хэш-функция неплохого качества. -Если передано несколько аргументов, то функция вычисляется по тем же правилам, с помощью комбинации по цепочке с использованием комбинатора из CityHash. -Например, так вы можете вычислить чексумму всей таблицы с точностью до порядка строк: `SELECT sum(cityHash64(*)) FROM table`. + +Генерирует 64-х битное значение [CityHash](https://github.com/google/cityhash). + +``` +cityHash64(par1,...) +``` + +Это не криптографическая хэш-функция. Она использует CityHash алгоритм для строковых параметров и зависящую от реализации быструю некриптографическую хэш-функцию для параметров с другими типами данных. Функция использует комбинатор CityHash для получения конечных результатов. + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть любого [поддерживаемого типа данных](../../data_types/index.md). + +**Возвращаемое значение** + +Значение хэша с типом данных [UInt64](../../data_types/int_uint.md). + +**Примеры** + +Пример вызова: + +```sql +SELECT cityHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS CityHash, toTypeName(CityHash) AS type +``` +```text +┌─────────────CityHash─┬─type───┐ +│ 12072650598913549138 │ UInt64 │ +└──────────────────────┴────────┘ +``` + +А вот так вы можете вычислить чексумму всей таблицы с точностью до порядка строк: + +```sql +SELECT groupBitXor(cityHash64(*)) FROM table +``` ## intHash32 + Вычисляет 32-битный хэш-код от целого числа любого типа. Это сравнительно быстрая некриптографическая хэш-функция среднего качества для чисел. ## intHash64 + Вычисляет 64-битный хэш-код от целого числа любого типа. Работает быстрее, чем intHash32. Качество среднее. @@ -45,15 +134,186 @@ SipHash - криптографическая хэш-функция. Работа ## SHA224 ## SHA256 + Вычисляет SHA-1, SHA-224, SHA-256 от строки и возвращает полученный набор байт в виде FixedString(20), FixedString(28), FixedString(32). Функция работает достаточно медленно (SHA-1 - примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 - примерно 2.2 миллионов). Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать. Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при SELECT-ах. ## URLHash(url\[, N\]) + Быстрая некриптографическая хэш-функция неплохого качества для строки, полученной из URL путём некоторой нормализации. -`URLHash(s)` - вычислить хэш от строки без одного завершающего символа `/`, `?` или `#` на конце, если такой там есть. -`URLHash(s, N)` - вычислить хэш от строки до N-го уровня в иерархии URL, без одного завершающего символа `/`, `?` или `#` на конце, если такой там есть. +`URLHash(s)` - вычислить хэш от строки без одного завершающего символа `/`, `?` или `#` на конце, если там такой есть. +`URLHash(s, N)` - вычислить хэш от строки до N-го уровня в иерархии URL, без одного завершающего символа `/`, `?` или `#` на конце, если там такой есть. Уровни аналогичные URLHierarchy. Функция специфична для Яндекс.Метрики. +## farmHash64 + +Генерирует 64-х битное значение [FarmHash](https://github.com/google/farmhash). + +``` +farmHash64(par1, ...) +``` + +Из всех [доступных методов](https://github.com/google/farmhash/blob/master/src/farmhash.h) функция использует `Hash64`. + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть любого [поддерживаемого типа данных](../../data_types/index.md). + +**Возвращаемое значение** + +Значение хэша с типом данных [UInt64](../../data_types/int_uint.md). + +**Пример** + +```sql +SELECT farmHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS FarmHash, toTypeName(FarmHash) AS type +``` + +```text +┌─────────────FarmHash─┬─type───┐ +│ 17790458267262532859 │ UInt64 │ +└──────────────────────┴────────┘ +``` + +## javaHash {#hash_functions-javahash} + +Вычисляет [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452) от строки. +Принимает аргумент типа String. Возвращает значение типа Int32. + +## hiveHash + +Вычисляет HiveHash от строки. +Принимает аргумент типа String. Возвращает значение типа Int32. +HiveHash — это результат [JavaHash](#hash_functions-javahash) с обнулённым битом знака числа. Функция используется в [Apache Hive](https://en.wikipedia.org/wiki/Apache_Hive) вплоть до версии 3.0. + +## metroHash64 + +Генерирует 64-х битное значение [MetroHash](http://www.jandrewrogers.com/2015/05/27/metrohash/). + +``` +metroHash64(par1, ...) +``` + +**Параметры** + +Функция принимает переменное число входных параметров. Параметры могут быть любого [поддерживаемого типа данных](../../data_types/index.md). + +**Возвращаемое значение** + +Значение хэша с типом данных [UInt64](../../data_types/int_uint.md). + +**Пример** + +```sql +SELECT metroHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS MetroHash, toTypeName(MetroHash) AS type +``` + +```text +┌────────────MetroHash─┬─type───┐ +│ 14235658766382344533 │ UInt64 │ +└──────────────────────┴────────┘ +``` + +## jumpConsistentHash + +Вычисляет JumpConsistentHash от значения типа UInt64. +Принимает аргумент типа UInt64. Возвращает значение типа Int32. +Дополнительные сведения смотрите по ссылке: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf) + +## murmurHash2_32, murmurHash2_64 + +Генерирует значение [MurmurHash2](https://github.com/aappleby/smhasher). + +``` +murmurHash2_32(par1, ...) +murmurHash2_64(par1, ...) +``` + +**Параметры** + +Обе функции принимают переменное число входных параметров. Параметры могут быть любого [поддерживаемого типа данных](../../data_types/index.md). + +**Возвращаемое значение** + +- Функция `murmurHash2_32` возвращает значение типа [UInt32](../../data_types/int_uint.md). +- Функция `murmurHash2_64` возвращает значение типа [UInt64](../../data_types/int_uint.md). + +**Пример** + +```sql +SELECT murmurHash2_64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS MurmurHash2, toTypeName(MurmurHash2) AS type +``` + +```text +┌──────────MurmurHash2─┬─type───┐ +│ 11832096901709403633 │ UInt64 │ +└──────────────────────┴────────┘ +``` + +## murmurHash3_32, murmurHash3_64 + +Генерирует значение [MurmurHash3](https://github.com/aappleby/smhasher). + +``` +murmurHash3_32(par1, ...) +murmurHash3_64(par1, ...) +``` + +**Параметры** + +Обе функции принимают переменное число входных параметров. Параметры могут быть любого [поддерживаемого типа данных](../../data_types/index.md). + +**Возвращаемое значение** + +- Функция `murmurHash3_32` возвращает значение типа [UInt32](../../data_types/int_uint.md). +- Функция `murmurHash3_64` возвращает значение типа [UInt64](../../data_types/int_uint.md). + +**Пример** + +```sql +SELECT murmurHash3_32(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:00')) AS MurmurHash3, toTypeName(MurmurHash3) AS type +``` + +```text +┌─MurmurHash3─┬─type───┐ +│ 2152717 │ UInt32 │ +└─────────────┴────────┘ +``` + +## murmurHash3_128 + +Генерирует значение [MurmurHash3](https://github.com/aappleby/smhasher). + +``` +murmurHash3_128( expr ) +``` + +**Параметры** + +- `expr` — [выражение](../syntax.md#syntax-expressions) возвращающее значение типа[String](../../data_types/string.md). + +**Возвращаемое значение** + +Хэш-значение типа [FixedString(16)](../../data_types/fixedstring.md). + +**Пример** + +```sql +SELECT murmurHash3_128('example_string') AS MurmurHash3, toTypeName(MurmurHash3) AS type +``` + +```text +┌─MurmurHash3──────┬─type────────────┐ +│ 6�1�4"S5KT�~~q │ FixedString(16) │ +└──────────────────┴─────────────────┘ +``` + +## xxHash32, xxHash64 + +Вычисляет xxHash от строки. +Принимает аргумент типа String. Возвращает значение типа Uint64 или Uint32. +Дополнительные сведения см. по ссылке: [xxHash](http://cyan4973.github.io/xxHash/) + [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/hash_functions/) diff --git a/docs/ru/query_language/functions/type_conversion_functions.md b/docs/ru/query_language/functions/type_conversion_functions.md index 5e25d767e08..8635ea089e0 100644 --- a/docs/ru/query_language/functions/type_conversion_functions.md +++ b/docs/ru/query_language/functions/type_conversion_functions.md @@ -184,7 +184,7 @@ SELECT toFixedString('foo\0bar', 8) AS s, toStringCutToZero(s) AS s_cut Функции принимают строку и интерпретируют байты, расположенные в начале строки, как число в host order (little endian). Если строка имеет недостаточную длину, то функции работают так, как будто строка дополнена необходимым количеством нулевых байт. Если строка длиннее, чем нужно, то лишние байты игнорируются. Дата интерпретируется, как число дней с начала unix-эпохи, а дата-с-временем - как число секунд с начала unix-эпохи. -## reinterpretAsString +## reinterpretAsString {#type_conversion_functions-reinterpretAsString} Функция принимает число или дату или дату-с-временем и возвращает строку, содержащую байты, представляющие соответствующее значение в host order (little endian). При этом, отбрасываются нулевые байты с конца. Например, значение 255 типа UInt32 будет строкой длины 1 байт. ## CAST(x, t) {#type_conversion_function-cast} diff --git a/docs/ru/query_language/misc.md b/docs/ru/query_language/misc.md index 942ce4a541d..93f548bf73c 100644 --- a/docs/ru/query_language/misc.md +++ b/docs/ru/query_language/misc.md @@ -30,15 +30,18 @@ CHECK TABLE [db.]name - 0 - данные в таблице повреждены; - 1 - данные не повреждены. -Запрос `CHECK TABLE` поддерживается только для следующих движков: +Запрос `CHECK TABLE` поддерживает следующие движки таблиц: - [Log](../operations/table_engines/log.md) - [TinyLog](../operations/table_engines/tinylog.md) -- StripeLog +- [StripeLog](../operations/table_engines/stripelog.md) +- [Семейство MergeTree](../operations/table_engines/mergetree.md) -В этих движках не предусмотрено автоматическое восстановление данных после сбоя. Поэтому используйте запрос `CHECK TABLE`, чтобы своевременно выявить повреждение данных. +При попытке выполнить запрос с таблицами с другими табличными движками, ClickHouse генерирует исключение. -Обратите внимание, высокая защита целостности данных обеспечивается в таблицах семейства [MergeTree](../operations/table_engines/mergetree.md). Для избежания потери данных рекомендуется использовать именно эти таблицы. +В движках `*Log` не предусмотрено автоматическое восстановление данных после сбоя. Используйте запрос `CHECK TABLE`, чтобы своевременно выявлять повреждение данных. + +Для движков из семейства `MergeTree` запрос `CHECK TABLE` показывает статус проверки для каждого отдельного куска данных таблицы на локальном сервере. **Что делать, если данные повреждены** @@ -56,7 +59,7 @@ DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` Возвращает описание столбцов таблицы. -Результат запроса содержит столбцы (все столбцы имеют тип String): +Результат запроса содержит столбцы (все столбцы имеют тип String): - `name` — имя столбца таблицы; - `type`— тип столбца; diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index eb77f43a37f..8ebd9bddc8d 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -25,6 +25,7 @@ add_library (common src/argsToConfig.cpp src/StackTrace.cpp src/Pipe.cpp + src/phdr_cache.cpp include/common/SimpleCache.h include/common/StackTrace.h @@ -51,6 +52,7 @@ add_library (common include/common/getThreadNumber.h include/common/sleep.h include/common/SimpleCache.h + include/common/phdr_cache.h include/ext/bit_cast.h include/ext/collection_cast.h diff --git a/libs/libcommon/include/common/SimpleCache.h b/libs/libcommon/include/common/SimpleCache.h index 2cf4348d0d7..57247de696a 100644 --- a/libs/libcommon/include/common/SimpleCache.h +++ b/libs/libcommon/include/common/SimpleCache.h @@ -26,7 +26,7 @@ private: using Result = typename function_traits::result; std::map cache; - std::mutex mutex; + mutable std::mutex mutex; public: template @@ -66,6 +66,12 @@ public: } } + size_t size() const + { + std::lock_guard lock(mutex); + return cache.size(); + } + void drop() { std::lock_guard lock(mutex); diff --git a/libs/libcommon/include/common/StackTrace.h b/libs/libcommon/include/common/StackTrace.h index be3d6438386..997730eb440 100644 --- a/libs/libcommon/include/common/StackTrace.h +++ b/libs/libcommon/include/common/StackTrace.h @@ -17,7 +17,7 @@ struct NoCapture }; /// Tries to capture current stack trace using libunwind or signal context -/// NOTE: All StackTrace constructors are signal safe +/// NOTE: StackTrace calculation is signal safe only if updatePHDRCache() was called beforehand. class StackTrace { public: @@ -53,7 +53,7 @@ protected: static std::string toStringImpl(const Frames & frames, size_t size); size_t size = 0; - Frames frames; + Frames frames{}; }; std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext_t & context); diff --git a/libs/libcommon/include/common/config_common.h.in b/libs/libcommon/include/common/config_common.h.in index 247afd87aea..c9a693a970a 100644 --- a/libs/libcommon/include/common/config_common.h.in +++ b/libs/libcommon/include/common/config_common.h.in @@ -7,4 +7,5 @@ #cmakedefine01 USE_READLINE #cmakedefine01 USE_LIBEDIT #cmakedefine01 HAVE_READLINE_HISTORY -#cmakedefine01 NOT_UNBUNDLED +#cmakedefine01 UNBUNDLED +#cmakedefine01 USE_INTERNAL_UNWIND_LIBRARY diff --git a/libs/libcommon/include/common/phdr_cache.h b/libs/libcommon/include/common/phdr_cache.h new file mode 100644 index 00000000000..d2854ece0bc --- /dev/null +++ b/libs/libcommon/include/common/phdr_cache.h @@ -0,0 +1,19 @@ +#pragma once + +/// This code was based on the code by Fedor Korotkiy (prime@yandex-team.ru) for YT product in Yandex. + +/** Collects all dl_phdr_info items and caches them in a static array. + * Also rewrites dl_iterate_phdr with a lock-free version which consults the above cache + * thus eliminating scalability bottleneck in C++ exception unwinding. + * As a drawback, this only works if no dynamic object unloading happens after this point. + * This function is thread-safe. You should call it to update cache after loading new shared libraries. + * Otherwise exception handling from dlopened libraries won't work (will call std::terminate immediately). + * + * NOTE: It is disabled with Thread Sanitizer because TSan can only use original "dl_iterate_phdr" function. + */ +void updatePHDRCache(); + +/** Check if "dl_iterate_phdr" will be lock-free + * to determine if some features like Query Profiler can be used. + */ +bool hasPHDRCache(); diff --git a/libs/libcommon/src/StackTrace.cpp b/libs/libcommon/src/StackTrace.cpp index 8aea884d004..8323a737fdf 100644 --- a/libs/libcommon/src/StackTrace.cpp +++ b/libs/libcommon/src/StackTrace.cpp @@ -208,7 +208,7 @@ size_t StackTrace::getSize() const return size; } -const StackTrace::Frames& StackTrace::getFrames() const +const StackTrace::Frames & StackTrace::getFrames() const { return frames; } diff --git a/libs/libcommon/src/phdr_cache.cpp b/libs/libcommon/src/phdr_cache.cpp new file mode 100644 index 00000000000..a072bae3636 --- /dev/null +++ b/libs/libcommon/src/phdr_cache.cpp @@ -0,0 +1,122 @@ +/// This code was based on the code by Fedor Korotkiy (prime@yandex-team.ru) for YT product in Yandex. + +#if defined(__has_feature) + #if __has_feature(address_sanitizer) + #define ADDRESS_SANITIZER 1 + #endif + #if __has_feature(thread_sanitizer) + #define THREAD_SANITIZER 1 + #endif +#else + #if defined(__SANITIZE_ADDRESS__) + #define ADDRESS_SANITIZER 1 + #endif + #if defined(__SANITIZE_THREAD__) + #define THREAD_SANITIZER 1 + #endif +#endif + +#if defined(__linux__) && !defined(THREAD_SANITIZER) + #define USE_PHDR_CACHE 1 +#endif + + +/// Thread Sanitizer uses dl_iterate_phdr function on initialization and fails if we provide our own. +#ifdef USE_PHDR_CACHE + +#include +#include +#include +#include +#include +#include + + +namespace +{ + +// This is adapted from +// https://github.com/scylladb/seastar/blob/master/core/exception_hacks.hh +// https://github.com/scylladb/seastar/blob/master/core/exception_hacks.cc + +using DLIterateFunction = int (*) (int (*callback) (dl_phdr_info * info, size_t size, void * data), void * data); + +DLIterateFunction getOriginalDLIteratePHDR() +{ + void * func = dlsym(RTLD_NEXT, "dl_iterate_phdr"); + if (!func) + throw std::runtime_error("Cannot find dl_iterate_phdr function with dlsym"); + return reinterpret_cast(func); +} + + +using PHDRCache = std::vector; +std::atomic phdr_cache {}; + +} + + +extern "C" +#ifndef __clang__ +[[gnu::visibility("default")]] +[[gnu::externally_visible]] +#endif +int dl_iterate_phdr(int (*callback) (dl_phdr_info * info, size_t size, void * data), void * data) +{ + auto current_phdr_cache = phdr_cache.load(); + if (!current_phdr_cache) + { + // Cache is not yet populated, pass through to the original function. + return getOriginalDLIteratePHDR()(callback, data); + } + + int result = 0; + for (auto & entry : *current_phdr_cache) + { + result = callback(&entry, offsetof(dl_phdr_info, dlpi_adds), data); + if (result != 0) + break; + } + return result; +} + + +extern "C" +{ +#ifdef ADDRESS_SANITIZER +void __lsan_ignore_object(const void *); +#else +void __lsan_ignore_object(const void *) {} +#endif +} + + +void updatePHDRCache() +{ + // Fill out ELF header cache for access without locking. + // This assumes no dynamic object loading/unloading after this point + + PHDRCache * new_phdr_cache = new PHDRCache; + getOriginalDLIteratePHDR()([] (dl_phdr_info * info, size_t /*size*/, void * data) + { + reinterpret_cast(data)->push_back(*info); + return 0; + }, new_phdr_cache); + phdr_cache.store(new_phdr_cache); + + /// Memory is intentionally leaked. + __lsan_ignore_object(new_phdr_cache); +} + + +bool hasPHDRCache() +{ + return phdr_cache.load() != nullptr; +} + +#else + +void updatePHDRCache() {} +bool hasPHDRCache() { return false; } + +#endif diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 1ba03168496..aa4993acead 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include #include @@ -501,6 +502,35 @@ void BaseDaemon::closeFDs() } } +namespace +{ +/// In debug version on Linux, increase oom score so that clickhouse is killed +/// first, instead of some service. Use a carefully chosen random score of 555: +/// the maximum is 1000, and chromium uses 300 for its tab processes. Ignore +/// whatever errors that occur, because it's just a debugging aid and we don't +/// care if it breaks. +#if defined(__linux__) && !defined(NDEBUG) +void debugIncreaseOOMScore() +{ + const std::string new_score = "555"; + try + { + DB::WriteBufferFromFile buf("/proc/self/oom_score_adj"); + buf.write(new_score.c_str(), new_score.size()); + } + catch (const Poco::Exception & e) + { + LOG_WARNING(&Logger::root(), "Failed to adjust OOM score: '" + + e.displayText() + "'."); + return; + } + LOG_INFO(&Logger::root(), "Set OOM score adjustment to " + new_score); +} +#else +void debugIncreaseOOMScore() {} +#endif +} + void BaseDaemon::initialize(Application & self) { closeFDs(); @@ -630,6 +660,7 @@ void BaseDaemon::initialize(Application & self) initializeTerminationAndSignalProcessing(); logRevision(); + debugIncreaseOOMScore(); for (const auto & key : DB::getMultipleKeysFromConfig(config(), "", "graphite")) { diff --git a/libs/libmysqlxx/include/mysqlxx/Pool.h b/libs/libmysqlxx/include/mysqlxx/Pool.h index 19b33a5c228..5261ffab017 100644 --- a/libs/libmysqlxx/include/mysqlxx/Pool.h +++ b/libs/libmysqlxx/include/mysqlxx/Pool.h @@ -72,25 +72,25 @@ public: return data == nullptr; } - operator mysqlxx::Connection & () + operator mysqlxx::Connection & () & { forceConnected(); return data->conn; } - operator const mysqlxx::Connection & () const + operator const mysqlxx::Connection & () const & { forceConnected(); return data->conn; } - const mysqlxx::Connection * operator->() const + const mysqlxx::Connection * operator->() const & { forceConnected(); return &data->conn; } - mysqlxx::Connection * operator->() + mysqlxx::Connection * operator->() & { forceConnected(); return &data->conn; diff --git a/utils/build/build_macos.sh b/utils/build/build_macos.sh index a9bf2481a0e..aa1b1a039b0 100755 --- a/utils/build/build_macos.sh +++ b/utils/build/build_macos.sh @@ -37,7 +37,7 @@ fi mkdir build cd build -cmake .. -DCMAKE_CXX_COMPILER=`which g++-8 g++-7` -DCMAKE_C_COMPILER=`which gcc-8 gcc-7` +cmake .. -DCMAKE_CXX_COMPILER=`which g++-9 g++-8 g++-7` -DCMAKE_C_COMPILER=`which gcc-9 gcc-8 gcc-7` cmake --build . cd .. diff --git a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp index 4acb8aae9b2..c956dea8712 100644 --- a/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp +++ b/utils/test-data-generator/ProtobufDelimitedMessagesSerializer.cpp @@ -9,7 +9,7 @@ #include "00825_protobuf_format_syntax2.pb.h" -void writeInsertQueryCommand(std::ostream & out, const std::string & format_schema, std::stringstream & delimited_messages) +void writeInsertDataQueryForInputTest(std::stringstream & delimited_messages, const std::string & table_name, const std::string & format_schema, std::ostream & out) { out << "echo -ne '"; std::string bytes = delimited_messages.str(); @@ -20,12 +20,12 @@ void writeInsertQueryCommand(std::ostream & out, const std::string & format_sche sprintf(buf, "\\x%02x", static_cast(c)); out << buf; } - out << "' | $CLICKHOUSE_CLIENT --query=\"INSERT INTO test.table FORMAT Protobuf" + out << "' | $CLICKHOUSE_CLIENT --query=\"INSERT INTO " << table_name << " FORMAT Protobuf" " SETTINGS format_schema = '$CURDIR/" << format_schema << "'\"" << std::endl; } -void writeInputInsertQueries(std::ostream & out) +void writeInsertDataQueriesForInputTest(std::ostream & out) { std::stringstream ss; { @@ -125,7 +125,7 @@ void writeInputInsertQueries(std::ostream & out) google::protobuf::util::SerializeDelimitedToOstream(person, &ss); } - writeInsertQueryCommand(out, "00825_protobuf_format:Person", ss); + writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format:Person", out); { AltPerson person; @@ -189,7 +189,7 @@ void writeInputInsertQueries(std::ostream & out) google::protobuf::util::SerializeDelimitedToOstream(person, &ss); } - writeInsertQueryCommand(out, "00825_protobuf_format:AltPerson", ss); + writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format:AltPerson", out); { StrPerson person; @@ -225,7 +225,7 @@ void writeInputInsertQueries(std::ostream & out) google::protobuf::util::SerializeDelimitedToOstream(person, &ss); } - writeInsertQueryCommand(out, "00825_protobuf_format:StrPerson", ss); + writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format:StrPerson", out); { Syntax2Person person; @@ -262,11 +262,34 @@ void writeInputInsertQueries(std::ostream & out) google::protobuf::util::SerializeDelimitedToOstream(person, &ss); } - writeInsertQueryCommand(out, "00825_protobuf_format_syntax2:Syntax2Person", ss); + writeInsertDataQueryForInputTest(ss, "in_persons_00825", "00825_protobuf_format_syntax2:Syntax2Person", out); + + { + NumberAndSquare ns; + ns.set_number(2); + ns.set_square(4); + google::protobuf::util::SerializeDelimitedToOstream(ns, &ss); + } + + { + NumberAndSquare ns; + ns.set_number(0); + ns.set_square(0); + google::protobuf::util::SerializeDelimitedToOstream(ns, &ss); + } + + { + NumberAndSquare ns; + ns.set_number(3); + ns.set_square(9); + google::protobuf::util::SerializeDelimitedToOstream(ns, &ss); + } + + writeInsertDataQueryForInputTest(ss, "in_squares_00825", "00825_protobuf_format:NumberAndSquare", out); } -void writeOutputReference(std::ostream & out) +void writeReferenceForOutputTest(std::ostream & out) { { Person person; @@ -637,6 +660,29 @@ void writeOutputReference(std::ostream & out) person.mutable_nestiness()->mutable_a()->mutable_b()->mutable_c()->set_d(503); google::protobuf::util::SerializeDelimitedToOstream(person, &out); } + + out << "SQUARES->" << std::endl; + + { + NumberAndSquare ns; + ns.set_number(0); + ns.set_square(0); + google::protobuf::util::SerializeDelimitedToOstream(ns, &out); + } + + { + NumberAndSquare ns; + ns.set_number(2); + ns.set_square(4); + google::protobuf::util::SerializeDelimitedToOstream(ns, &out); + } + + { + NumberAndSquare ns; + ns.set_number(3); + ns.set_square(9); + google::protobuf::util::SerializeDelimitedToOstream(ns, &out); + } } @@ -676,7 +722,7 @@ int main(int argc, char ** argv) { std::string output_dir; parseCommandLine(argc, argv, output_dir); - writeFile(output_dir + "/00825_protobuf_format_input.insh", writeInputInsertQueries); - writeFile(output_dir + "/00825_protobuf_format_output.reference", writeOutputReference); + writeFile(output_dir + "/00825_protobuf_format_input.insh", writeInsertDataQueriesForInputTest); + writeFile(output_dir + "/00825_protobuf_format_output.reference", writeReferenceForOutputTest); return 0; }