diff --git a/README.md b/README.md index 06ea8e94592..ae4abb10941 100644 --- a/README.md +++ b/README.md @@ -15,4 +15,4 @@ ClickHouse is an open-source column-oriented database management system that all ## Upcoming Events * [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3. - +* [ClickHouse Meetup in Moscow](https://yandex.ru/promo/clickhouse/moscow-december-2019) on December 11. diff --git a/contrib/libunwind-cmake/CMakeLists.txt b/contrib/libunwind-cmake/CMakeLists.txt index f09d0979692..7a6648d8dc6 100644 --- a/contrib/libunwind-cmake/CMakeLists.txt +++ b/contrib/libunwind-cmake/CMakeLists.txt @@ -11,7 +11,9 @@ endif () set(LIBUNWIND_C_SOURCES ${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1.c ${LIBUNWIND_SOURCE_DIR}/src/UnwindLevel1-gcc-ext.c - ${LIBUNWIND_SOURCE_DIR}/src/Unwind-sjlj.c) + ${LIBUNWIND_SOURCE_DIR}/src/Unwind-sjlj.c + # Use unw_backtrace to override libgcc's backtrace symbol for better ABI compatibility + unwind-override.c) set_source_files_properties(${LIBUNWIND_C_SOURCES} PROPERTIES COMPILE_FLAGS "-std=c99") set(LIBUNWIND_ASM_SOURCES diff --git a/contrib/libunwind-cmake/unwind-override.c b/contrib/libunwind-cmake/unwind-override.c new file mode 100644 index 00000000000..616bab6ae4b --- /dev/null +++ b/contrib/libunwind-cmake/unwind-override.c @@ -0,0 +1,6 @@ +#include + +int backtrace(void ** buffer, int size) +{ + return unw_backtrace(buffer, size); +} diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 510faed187b..fecc1fa7e76 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -376,6 +376,10 @@ if (USE_POCO_MONGODB) dbms_target_link_libraries (PRIVATE ${Poco_MongoDB_LIBRARY}) endif() +if (USE_POCO_REDIS) + dbms_target_link_libraries (PRIVATE ${Poco_Redis_LIBRARY}) +endif() + if (USE_POCO_NETSSL) target_link_libraries (clickhouse_common_io PRIVATE ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY}) dbms_target_link_libraries (PRIVATE ${Poco_NetSSL_LIBRARY} ${Poco_Crypto_LIBRARY}) diff --git a/dbms/programs/server/MySQLHandler.cpp b/dbms/programs/server/MySQLHandler.cpp index 56ec25686dd..c2de9eb74e0 100644 --- a/dbms/programs/server/MySQLHandler.cpp +++ b/dbms/programs/server/MySQLHandler.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #if USE_POCO_NETSSL #include @@ -267,29 +268,49 @@ void MySQLHandler::comPing() packet_sender->sendPacket(OK_Packet(0x0, client_capability_flags, 0, 0, 0), true); } +static bool isFederatedServerSetupCommand(const String & query); + void MySQLHandler::comQuery(ReadBuffer & payload) { - bool with_output = false; - std::function set_content_type = [&with_output](const String &) -> void { - with_output = true; - }; + String query = String(payload.position(), payload.buffer().end()); - const String query("select ''"); - ReadBufferFromString empty_select(query); - - bool should_replace = false; - // Translate query from MySQL to ClickHouse. - // This is a temporary workaround until ClickHouse supports the syntax "@@var_name". - if (std::string(payload.position(), payload.buffer().end()) == "select @@version_comment limit 1") // MariaDB client starts session with that query + // This is a workaround in order to support adding ClickHouse to MySQL using federated server. + // As Clickhouse doesn't support these statements, we just send OK packet in response. + if (isFederatedServerSetupCommand(query)) { - should_replace = true; - } - - Context query_context = connection_context; - executeQuery(should_replace ? empty_select : payload, *out, true, query_context, set_content_type, nullptr); - - if (!with_output) packet_sender->sendPacket(OK_Packet(0x00, client_capability_flags, 0, 0, 0), true); + } + else + { + bool with_output = false; + std::function set_content_type = [&with_output](const String &) -> void { + with_output = true; + }; + + String replacement_query = "select ''"; + bool should_replace = false; + + // Translate query from MySQL to ClickHouse. + // This is a temporary workaround until ClickHouse supports the syntax "@@var_name". + if (query == "select @@version_comment limit 1") // MariaDB client starts session with that query + { + should_replace = true; + } + // This is a workaround in order to support adding ClickHouse to MySQL using federated server. + if (0 == strncasecmp("SHOW TABLE STATUS LIKE", query.c_str(), 22)) + { + should_replace = true; + replacement_query = boost::replace_all_copy(query, "SHOW TABLE STATUS LIKE ", show_table_status_replacement_query); + } + + ReadBufferFromString replacement(replacement_query); + + Context query_context = connection_context; + executeQuery(should_replace ? replacement : payload, *out, true, query_context, set_content_type, nullptr); + + if (!with_output) + packet_sender->sendPacket(OK_Packet(0x00, client_capability_flags, 0, 0, 0), true); + } } void MySQLHandler::authPluginSSL() @@ -335,4 +356,33 @@ void MySQLHandlerSSL::finishHandshakeSSL(size_t packet_size, char * buf, size_t #endif +static bool isFederatedServerSetupCommand(const String & query) +{ + return 0 == strncasecmp("SET NAMES", query.c_str(), 9) || 0 == strncasecmp("SET character_set_results", query.c_str(), 25) + || 0 == strncasecmp("SET FOREIGN_KEY_CHECKS", query.c_str(), 22) || 0 == strncasecmp("SET AUTOCOMMIT", query.c_str(), 14) + || 0 == strncasecmp("SET SESSION TRANSACTION ISOLATION LEVEL", query.c_str(), 39); +} + +const String MySQLHandler::show_table_status_replacement_query("SELECT" + " name AS Name," + " engine AS Engine," + " '10' AS Version," + " 'Dynamic' AS Row_format," + " 0 AS Rows," + " 0 AS Avg_row_length," + " 0 AS Data_length," + " 0 AS Max_data_length," + " 0 AS Index_length," + " 0 AS Data_free," + " 'NULL' AS Auto_increment," + " metadata_modification_time AS Create_time," + " metadata_modification_time AS Update_time," + " metadata_modification_time AS Check_time," + " 'utf8_bin' AS Collation," + " 'NULL' AS Checksum," + " '' AS Create_options," + " '' AS Comment" + " FROM system.tables" + " WHERE name LIKE "); + } diff --git a/dbms/programs/server/MySQLHandler.h b/dbms/programs/server/MySQLHandler.h index 96cb353d897..42629470632 100644 --- a/dbms/programs/server/MySQLHandler.h +++ b/dbms/programs/server/MySQLHandler.h @@ -11,7 +11,6 @@ namespace DB { - /// Handler for MySQL wire protocol connections. Allows to connect to ClickHouse using MySQL client. class MySQLHandler : public Poco::Net::TCPServerConnection { @@ -59,6 +58,9 @@ protected: std::shared_ptr out; bool secure_connection = false; + +private: + static const String show_table_status_replacement_query; }; #if USE_SSL && USE_POCO_NETSSL diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 5da6cc59527..5fdf9db1ab2 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -219,6 +219,7 @@ public: Field getField() const { return getDataColumn()[0]; } + /// The constant value. It is valid even if the size of the column is 0. template T getValue() const { return getField().safeGet>(); } }; diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index ad9d00661a0..5c6f7f0fdd5 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -144,7 +144,7 @@ public: } - void insert(const T value) { data.push_back(value); } + void insertValue(const T value) { data.push_back(value); } Container & getData() { return data; } const Container & getData() const { return data; } const T & getElement(size_t n) const { return data[n]; } diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 7abc7dc9232..4f49ca92df4 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -469,7 +469,6 @@ namespace ErrorCodes extern const int POCO_EXCEPTION = 1000; extern const int STD_EXCEPTION = 1001; extern const int UNKNOWN_EXCEPTION = 1002; - extern const int METRIKA_OTHER_ERROR = 1003; extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND = 2001; extern const int ILLEGAL_PROJECTION_MANIPULATOR = 2002; diff --git a/dbms/src/Common/Exception.h b/dbms/src/Common/Exception.h index bd4d6e0be09..5df2879a16d 100644 --- a/dbms/src/Common/Exception.h +++ b/dbms/src/Common/Exception.h @@ -17,7 +17,6 @@ namespace DB namespace ErrorCodes { extern const int POCO_EXCEPTION; - extern const int METRIKA_OTHER_ERROR; } class Exception : public Poco::Exception diff --git a/dbms/src/Common/HashTable/Hash.h b/dbms/src/Common/HashTable/Hash.h index 90ee89953c0..befb660a968 100644 --- a/dbms/src/Common/HashTable/Hash.h +++ b/dbms/src/Common/HashTable/Hash.h @@ -84,6 +84,23 @@ struct DefaultHash>> } }; +template +struct DefaultHash && sizeof(T) <= 8>> +{ + size_t operator() (T key) const + { + return DefaultHash64(key); + } +}; + +template +struct DefaultHash && sizeof(T) == 16>> +{ + size_t operator() (T key) const + { + return DefaultHash64(key >> 64) ^ DefaultHash64(key); + } +}; template struct HashCRC32; diff --git a/dbms/src/Common/StackTrace.cpp b/dbms/src/Common/StackTrace.cpp index 4ab0847ac18..2f3c4e9c2fa 100644 --- a/dbms/src/Common/StackTrace.cpp +++ b/dbms/src/Common/StackTrace.cpp @@ -158,7 +158,7 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext break; } - case SIGPROF: + case SIGTSTP: { error << "This is a signal used for debugging purposes by the user."; break; diff --git a/dbms/src/Core/MySQLProtocol.cpp b/dbms/src/Core/MySQLProtocol.cpp index 1c4e94c492c..12fd6f963a1 100644 --- a/dbms/src/Core/MySQLProtocol.cpp +++ b/dbms/src/Core/MySQLProtocol.cpp @@ -100,4 +100,71 @@ size_t getLengthEncodedStringSize(const String & s) return getLengthEncodedNumberSize(s.size()) + s.size(); } +ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex type_index) +{ + ColumnType column_type; + int flags = 0; + switch (type_index) + { + case TypeIndex::UInt8: + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt16: + column_type = ColumnType::MYSQL_TYPE_SHORT; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt32: + column_type = ColumnType::MYSQL_TYPE_LONG; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::UInt64: + column_type = ColumnType::MYSQL_TYPE_LONGLONG; + flags = ColumnDefinitionFlags::BINARY_FLAG | ColumnDefinitionFlags::UNSIGNED_FLAG; + break; + case TypeIndex::Int8: + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int16: + column_type = ColumnType::MYSQL_TYPE_SHORT; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int32: + column_type = ColumnType::MYSQL_TYPE_LONG; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Int64: + column_type = ColumnType::MYSQL_TYPE_LONGLONG; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Float32: + column_type = ColumnType::MYSQL_TYPE_FLOAT; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Float64: + column_type = ColumnType::MYSQL_TYPE_TINY; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::Date: + column_type = ColumnType::MYSQL_TYPE_DATE; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::DateTime: + column_type = ColumnType::MYSQL_TYPE_DATETIME; + flags = ColumnDefinitionFlags::BINARY_FLAG; + break; + case TypeIndex::String: + column_type = ColumnType::MYSQL_TYPE_STRING; + break; + case TypeIndex::FixedString: + column_type = ColumnType::MYSQL_TYPE_STRING; + break; + default: + column_type = ColumnType::MYSQL_TYPE_STRING; + break; + } + return ColumnDefinition(column_name, CharacterSet::binary, 0, column_type, flags, 0); +} + } diff --git a/dbms/src/Core/MySQLProtocol.h b/dbms/src/Core/MySQLProtocol.h index 5e8be549bbd..db7a8dae2fa 100644 --- a/dbms/src/Core/MySQLProtocol.h +++ b/dbms/src/Core/MySQLProtocol.h @@ -130,6 +130,14 @@ enum ColumnType }; +// https://dev.mysql.com/doc/dev/mysql-server/latest/group__group__cs__column__definition__flags.html +enum ColumnDefinitionFlags +{ + UNSIGNED_FLAG = 32, + BINARY_FLAG = 128 +}; + + class ProtocolError : public DB::Exception { public: @@ -824,19 +832,40 @@ protected: } }; + +ColumnDefinition getColumnDefinition(const String & column_name, const TypeIndex index); + + +namespace ProtocolText +{ + class ResultsetRow : public WritePacket { - std::vector columns; + const Columns & columns; + int row_num; size_t payload_size = 0; + std::vector serialized; public: - ResultsetRow() = default; - - void appendColumn(String && value) + ResultsetRow(const DataTypes & data_types, const Columns & columns_, int row_num_) + : columns(columns_) + , row_num(row_num_) { - payload_size += getLengthEncodedStringSize(value); - columns.emplace_back(std::move(value)); + for (size_t i = 0; i < columns.size(); i++) + { + if (columns[i]->isNullAt(row_num)) + { + payload_size += 1; + serialized.emplace_back("\xfb"); + } + else + { + WriteBufferFromOwnString ostr; + data_types[i]->serializeAsText(*columns[i], row_num, ostr, FormatSettings()); + payload_size += getLengthEncodedStringSize(ostr.str()); + serialized.push_back(std::move(ostr.str())); + } + } } - protected: size_t getPayloadSize() const override { @@ -845,11 +874,18 @@ protected: void writePayloadImpl(WriteBuffer & buffer) const override { - for (const String & column : columns) - writeLengthEncodedString(column, buffer); + for (size_t i = 0; i < columns.size(); i++) + { + if (columns[i]->isNullAt(row_num)) + buffer.write(serialized[i].data(), 1); + else + writeLengthEncodedString(serialized[i], buffer); + } } }; +} + namespace Authentication { diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 06b77e22ca5..00d7efb4a5b 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -76,6 +76,7 @@ struct Settings : public SettingsCollection M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ + M(SettingUInt64, background_move_pool_size, 8, "Number of threads performing background moves for tables. Only has meaning at server startup.", 0) \ M(SettingUInt64, background_schedule_pool_size, 16, "Number of threads performing background tasks for replicated tables. Only has meaning at server startup.", 0) \ \ M(SettingMilliseconds, distributed_directory_monitor_sleep_time_ms, 100, "Sleep time for StorageDistributed DirectoryMonitors, in case of any errors delay grows exponentially.", 0) \ diff --git a/dbms/src/Core/TypeListNumber.h b/dbms/src/Core/TypeListNumber.h index d9e6f82a7a6..84b716fa5b8 100644 --- a/dbms/src/Core/TypeListNumber.h +++ b/dbms/src/Core/TypeListNumber.h @@ -5,6 +5,9 @@ namespace DB { -using TypeListNumbers = TypeList; +using TypeListNativeNumbers = TypeList; +using TypeListDecimalNumbers = TypeList; +using TypeListNumbers = TypeList; } diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 1c73456154a..f8742d26ad9 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -57,6 +57,13 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server } } +void NativeBlockInputStream::resetParser() +{ + istr_concrete = nullptr; + use_index = false; + header.clear(); + avg_value_size_hints.clear(); +} void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) { diff --git a/dbms/src/DataStreams/NativeBlockInputStream.h b/dbms/src/DataStreams/NativeBlockInputStream.h index 0502d077e3a..774a1cfa1cd 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.h +++ b/dbms/src/DataStreams/NativeBlockInputStream.h @@ -78,6 +78,9 @@ public: Block getHeader() const override; + void resetParser(); + + protected: Block readImpl() override; diff --git a/dbms/src/DataTypes/DataTypeLowCardinality.cpp b/dbms/src/DataTypes/DataTypeLowCardinality.cpp index 362db4efa33..417c988e5b9 100644 --- a/dbms/src/DataTypes/DataTypeLowCardinality.cpp +++ b/dbms/src/DataTypes/DataTypeLowCardinality.cpp @@ -894,7 +894,7 @@ MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDat if (isColumnedAsNumber(type)) { MutableColumnUniquePtr column; - TypeListNumbers::forEach(CreateColumnVector(column, *type, creator)); + TypeListNativeNumbers::forEach(CreateColumnVector(column, *type, creator)); if (!column) throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR); diff --git a/dbms/src/Databases/DatabaseLazy.cpp b/dbms/src/Databases/DatabaseLazy.cpp index d84ee61aea5..b232bf49392 100644 --- a/dbms/src/Databases/DatabaseLazy.cpp +++ b/dbms/src/Databases/DatabaseLazy.cpp @@ -361,9 +361,8 @@ StoragePtr DatabaseLazy::loadTable(const Context & context, const String & table } catch (const Exception & e) { - throw Exception("Cannot create table from metadata file " + table_metadata_path + ", error: " + e.displayText() + - ", stack trace:\n" + e.getStackTrace().toString(), - ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA); + throw Exception("Cannot create table from metadata file " + table_metadata_path + ". Error: " + DB::getCurrentExceptionMessage(true), + e, DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA); } } diff --git a/dbms/src/Databases/DatabaseOrdinary.cpp b/dbms/src/Databases/DatabaseOrdinary.cpp index a50ad4615e4..57138f8237f 100644 --- a/dbms/src/Databases/DatabaseOrdinary.cpp +++ b/dbms/src/Databases/DatabaseOrdinary.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include #include @@ -81,9 +82,8 @@ try catch (const Exception & e) { throw Exception( - "Cannot create object '" + query.table + "' from query " + serializeAST(query) + ", error: " + e.displayText() + ", stack trace:\n" - + e.getStackTrace().toString(), - ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA); + "Cannot create object '" + query.table + "' from query " + serializeAST(query) + ". Error: " + DB::getCurrentExceptionMessage(true), + e, DB::ErrorCodes::CANNOT_CREATE_TABLE_FROM_METADATA); } @@ -138,8 +138,7 @@ void DatabaseOrdinary::loadStoredObjects( catch (const Exception & e) { throw Exception( - "Cannot parse definition from metadata file " + full_path + ", error: " + e.displayText() + ", stack trace:\n" - + e.getStackTrace().toString(), ErrorCodes::CANNOT_PARSE_TEXT); + "Cannot parse definition from metadata file " + full_path + ". Error: " + DB::getCurrentExceptionMessage(true), e, ErrorCodes::CANNOT_PARSE_TEXT); } }); @@ -180,7 +179,15 @@ void DatabaseOrdinary::loadStoredObjects( auto & external_loader = context.getExternalDictionariesLoader(); external_loader.addConfigRepository(getDatabaseName(), std::move(dictionaries_repository)); bool lazy_load = context.getConfigRef().getBool("dictionaries_lazy_load", true); - external_loader.reload(!lazy_load); + + auto filter = [this](const std::string & dictionary_name) -> bool + { + if (!startsWith(dictionary_name, name + "." /* db name */)) + return false; + LOG_INFO(log, "Loading dictionary " << backQuote(dictionary_name) << ", for database " << backQuote(name)); + return true; + }; + external_loader.reload(filter, !lazy_load); } diff --git a/dbms/src/Dictionaries/CacheDictionary.h b/dbms/src/Dictionaries/CacheDictionary.h index 750c51a7cf3..b5065a63922 100644 --- a/dbms/src/Dictionaries/CacheDictionary.h +++ b/dbms/src/Dictionaries/CacheDictionary.h @@ -48,7 +48,7 @@ public: double getLoadFactor() const override { return static_cast(element_count.load(std::memory_order_relaxed)) / size; } - bool isCached() const override { return true; } + bool supportUpdates() const override { return false; } std::shared_ptr clone() const override { diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index d8146548c2b..e9269cb165a 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -71,7 +71,7 @@ public: double getLoadFactor() const override { return static_cast(element_count.load(std::memory_order_relaxed)) / size; } - bool isCached() const override { return true; } + bool supportUpdates() const override { return false; } std::shared_ptr clone() const override { diff --git a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h index 68b8d9d0d36..77941d6c5df 100644 --- a/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyHashedDictionary.h @@ -46,8 +46,6 @@ public: double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - bool isCached() const override { return false; } - std::shared_ptr clone() const override { return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); diff --git a/dbms/src/Dictionaries/FlatDictionary.h b/dbms/src/Dictionaries/FlatDictionary.h index d9ea141de2e..1bb06348aab 100644 --- a/dbms/src/Dictionaries/FlatDictionary.h +++ b/dbms/src/Dictionaries/FlatDictionary.h @@ -43,8 +43,6 @@ public: double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - bool isCached() const override { return false; } - std::shared_ptr clone() const override { return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, saved_block); diff --git a/dbms/src/Dictionaries/HashedDictionary.h b/dbms/src/Dictionaries/HashedDictionary.h index b605157eb93..d4f55dc8e39 100644 --- a/dbms/src/Dictionaries/HashedDictionary.h +++ b/dbms/src/Dictionaries/HashedDictionary.h @@ -48,8 +48,6 @@ public: double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - bool isCached() const override { return false; } - std::shared_ptr clone() const override { return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty, sparse, saved_block); diff --git a/dbms/src/Dictionaries/IDictionary.h b/dbms/src/Dictionaries/IDictionary.h index 9ce7c569f75..9c74c98e88a 100644 --- a/dbms/src/Dictionaries/IDictionary.h +++ b/dbms/src/Dictionaries/IDictionary.h @@ -37,8 +37,6 @@ struct IDictionaryBase : public IExternalLoadable virtual double getLoadFactor() const = 0; - virtual bool isCached() const = 0; - virtual const IDictionarySource * getSource() const = 0; virtual const DictionaryStructure & getStructure() const = 0; @@ -47,7 +45,7 @@ struct IDictionaryBase : public IExternalLoadable virtual BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const = 0; - bool supportUpdates() const override { return !isCached(); } + bool supportUpdates() const override { return true; } bool isModified() const override { diff --git a/dbms/src/Dictionaries/RangeHashedDictionary.h b/dbms/src/Dictionaries/RangeHashedDictionary.h index 6e03fc30720..829553c68b3 100644 --- a/dbms/src/Dictionaries/RangeHashedDictionary.h +++ b/dbms/src/Dictionaries/RangeHashedDictionary.h @@ -38,8 +38,6 @@ public: double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - bool isCached() const override { return false; } - std::shared_ptr clone() const override { return std::make_shared(dictionary_name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); diff --git a/dbms/src/Dictionaries/TrieDictionary.h b/dbms/src/Dictionaries/TrieDictionary.h index 18b1b1c79b9..7e41942b873 100644 --- a/dbms/src/Dictionaries/TrieDictionary.h +++ b/dbms/src/Dictionaries/TrieDictionary.h @@ -47,8 +47,6 @@ public: double getLoadFactor() const override { return static_cast(element_count) / bucket_count; } - bool isCached() const override { return false; } - std::shared_ptr clone() const override { return std::make_shared(name, dict_struct, source_ptr->clone(), dict_lifetime, require_nonempty); diff --git a/dbms/src/Functions/GatherUtils/Algorithms.h b/dbms/src/Functions/GatherUtils/Algorithms.h index c4b21ced4ae..9904d0619f1 100644 --- a/dbms/src/Functions/GatherUtils/Algorithms.h +++ b/dbms/src/Functions/GatherUtils/Algorithms.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include "Sources.h" #include "Sinks.h" @@ -79,8 +80,16 @@ inline ALWAYS_INLINE void writeSlice(const NumericArraySlice & slice, Generic { for (size_t i = 0; i < slice.size; ++i) { - Field field = T(slice.data[i]); - sink.elements.insert(field); + if constexpr (IsDecimalNumber) + { + DecimalField field(T(slice.data[i]), 0); /// TODO: Decimal scale + sink.elements.insert(field); + } + else + { + Field field = T(slice.data[i]); + sink.elements.insert(field); + } } sink.current_offset += slice.size; } @@ -422,9 +431,18 @@ bool sliceHasImpl(const FirstSliceType & first, const SecondSliceType & second, } template -bool sliceEqualElements(const NumericArraySlice & first, const NumericArraySlice & second, size_t first_ind, size_t second_ind) +bool sliceEqualElements(const NumericArraySlice & first [[maybe_unused]], + const NumericArraySlice & second [[maybe_unused]], + size_t first_ind [[maybe_unused]], + size_t second_ind [[maybe_unused]]) { - return accurate::equalsOp(first.data[first_ind], second.data[second_ind]); + /// TODO: Decimal scale + if constexpr (IsDecimalNumber && IsDecimalNumber) + return accurate::equalsOp(typename T::NativeType(first.data[first_ind]), typename U::NativeType(second.data[second_ind])); + else if constexpr (IsDecimalNumber || IsDecimalNumber) + return false; + else + return accurate::equalsOp(first.data[first_ind], second.data[second_ind]); } template diff --git a/dbms/src/Functions/GatherUtils/Sinks.h b/dbms/src/Functions/GatherUtils/Sinks.h index c6925fab865..5fd943ae78b 100644 --- a/dbms/src/Functions/GatherUtils/Sinks.h +++ b/dbms/src/Functions/GatherUtils/Sinks.h @@ -3,6 +3,7 @@ #include "IArraySink.h" #include +#include #include #include #include @@ -33,17 +34,18 @@ struct NullableValueSource; template struct NumericArraySink : public ArraySinkImpl> { + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; using CompatibleArraySource = NumericArraySource; using CompatibleValueSource = NumericValueSource; - typename ColumnVector::Container & elements; + typename ColVecType::Container & elements; typename ColumnArray::Offsets & offsets; size_t row_num = 0; ColumnArray::Offset current_offset = 0; NumericArraySink(ColumnArray & arr, size_t column_size) - : elements(typeid_cast &>(arr.getData()).getData()), offsets(arr.getOffsets()) + : elements(typeid_cast(arr.getData()).getData()), offsets(arr.getOffsets()) { offsets.resize(column_size); } diff --git a/dbms/src/Functions/GatherUtils/Sources.h b/dbms/src/Functions/GatherUtils/Sources.h index d43dc69b2b0..c21a6fc523c 100644 --- a/dbms/src/Functions/GatherUtils/Sources.h +++ b/dbms/src/Functions/GatherUtils/Sources.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -30,17 +31,18 @@ namespace GatherUtils template struct NumericArraySource : public ArraySourceImpl> { + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; using Slice = NumericArraySlice; using Column = ColumnArray; - const typename ColumnVector::Container & elements; + const typename ColVecType::Container & elements; const typename ColumnArray::Offsets & offsets; size_t row_num = 0; ColumnArray::Offset prev_offset = 0; explicit NumericArraySource(const ColumnArray & arr) - : elements(typeid_cast &>(arr.getData()).getData()), offsets(arr.getOffsets()) + : elements(typeid_cast(arr.getData()).getData()), offsets(arr.getOffsets()) { } @@ -650,7 +652,7 @@ template struct NumericValueSource : ValueSourceImpl> { using Slice = NumericValueSlice; - using Column = ColumnVector; + using Column = std::conditional_t, ColumnDecimal, ColumnVector>; const T * begin; size_t total_rows; diff --git a/dbms/src/Functions/GatherUtils/createArraySink.cpp b/dbms/src/Functions/GatherUtils/createArraySink.cpp index 0f052856dbe..e6d80cdab9f 100644 --- a/dbms/src/Functions/GatherUtils/createArraySink.cpp +++ b/dbms/src/Functions/GatherUtils/createArraySink.cpp @@ -14,7 +14,9 @@ struct ArraySinkCreator { static std::unique_ptr create(ColumnArray & col, NullMap * null_map, size_t column_size) { - if (typeid_cast *>(&col.getData())) + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + + if (typeid_cast(&col.getData())) { if (null_map) return std::make_unique>>(col, *null_map, column_size); diff --git a/dbms/src/Functions/GatherUtils/createArraySource.cpp b/dbms/src/Functions/GatherUtils/createArraySource.cpp index 2b0df7c7b7f..b7690a3f53c 100644 --- a/dbms/src/Functions/GatherUtils/createArraySource.cpp +++ b/dbms/src/Functions/GatherUtils/createArraySource.cpp @@ -14,7 +14,9 @@ struct ArraySourceCreator { static std::unique_ptr create(const ColumnArray & col, const NullMap * null_map, bool is_const, size_t total_rows) { - if (typeid_cast *>(&col.getData())) + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + + if (typeid_cast(&col.getData())) { if (null_map) { diff --git a/dbms/src/Functions/GatherUtils/createValueSource.cpp b/dbms/src/Functions/GatherUtils/createValueSource.cpp index faf7d96c4c9..c74c41999aa 100644 --- a/dbms/src/Functions/GatherUtils/createValueSource.cpp +++ b/dbms/src/Functions/GatherUtils/createValueSource.cpp @@ -14,7 +14,9 @@ struct ValueSourceCreator { static std::unique_ptr create(const IColumn & col, const NullMap * null_map, bool is_const, size_t total_rows) { - if (auto column_vector = typeid_cast *>(&col)) + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + + if (auto column_vector = typeid_cast(&col)) { if (null_map) { diff --git a/dbms/src/Functions/GeoUtils.h b/dbms/src/Functions/GeoUtils.h index 2191290d858..b13faa0f014 100644 --- a/dbms/src/Functions/GeoUtils.h +++ b/dbms/src/Functions/GeoUtils.h @@ -590,7 +590,7 @@ struct CallPointInPolygon template static ColumnPtr call(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl) { - using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNumbers>::Type; + using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNativeNumbers>::Type; if (auto column = typeid_cast *>(&x)) return Impl::template call(*column, y, impl); return CallPointInPolygon::call(x, y, impl); @@ -616,7 +616,7 @@ struct CallPointInPolygon<> template ColumnPtr pointInPolygon(const IColumn & x, const IColumn & y, PointInPolygonImpl && impl) { - using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNumbers>::Type; + using Impl = typename ApplyTypeListForClass<::DB::GeoUtils::CallPointInPolygon, TypeListNativeNumbers>::Type; return Impl::call(x, y, impl); } diff --git a/dbms/src/Functions/array/arrayCompact.cpp b/dbms/src/Functions/array/arrayCompact.cpp index b4e11a42f1a..489d18440e0 100644 --- a/dbms/src/Functions/array/arrayCompact.cpp +++ b/dbms/src/Functions/array/arrayCompact.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include #include @@ -27,16 +29,23 @@ struct ArrayCompactImpl template static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr) { - const ColumnVector * src_values_column = checkAndGetColumn>(mapped.get()); + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + + const ColVecType * src_values_column = checkAndGetColumn(mapped.get()); if (!src_values_column) return false; const IColumn::Offsets & src_offsets = array.getOffsets(); - const typename ColumnVector::Container & src_values = src_values_column->getData(); + const typename ColVecType::Container & src_values = src_values_column->getData(); - auto res_values_column = ColumnVector::create(src_values.size()); - typename ColumnVector::Container & res_values = res_values_column->getData(); + typename ColVecType::MutablePtr res_values_column; + if constexpr (IsDecimalNumber) + res_values_column = ColVecType::create(src_values.size(), src_values.getScale()); + else + res_values_column = ColVecType::create(src_values.size()); + + typename ColVecType::Container & res_values = res_values_column->getData(); size_t src_offsets_size = src_offsets.size(); auto res_offsets_column = ColumnArray::ColumnOffsets::create(src_offsets_size); IColumn::Offsets & res_offsets = res_offsets_column->getData(); @@ -129,7 +138,10 @@ struct ArrayCompactImpl executeType< Int32 >(mapped, array, res) || executeType< Int64 >(mapped, array, res) || executeType(mapped, array, res) || - executeType(mapped, array, res))) + executeType(mapped, array, res)) || + executeType(mapped, array, res) || + executeType(mapped, array, res) || + executeType(mapped, array, res)) { executeGeneric(mapped, array, res); } diff --git a/dbms/src/Functions/array/arrayCumSum.cpp b/dbms/src/Functions/array/arrayCumSum.cpp index 0649558c650..8a23a6609b4 100644 --- a/dbms/src/Functions/array/arrayCumSum.cpp +++ b/dbms/src/Functions/array/arrayCumSum.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include "FunctionArrayMapped.h" #include @@ -31,6 +33,13 @@ struct ArrayCumSumImpl if (which.isFloat()) return std::make_shared(std::make_shared()); + if (which.isDecimal()) + { + UInt32 scale = getDecimalScale(*expression_return); + DataTypePtr nested = std::make_shared>(maxDecimalPrecision(), scale); + return std::make_shared(nested); + } + throw Exception("arrayCumSum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -38,11 +47,14 @@ struct ArrayCumSumImpl template static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr) { - const ColumnVector * column = checkAndGetColumn>(&*mapped); + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + + const ColVecType * column = checkAndGetColumn(&*mapped); if (!column) { - const ColumnConst * column_const = checkAndGetColumnConst>(&*mapped); + const ColumnConst * column_const = checkAndGetColumnConst(&*mapped); if (!column_const) return false; @@ -50,8 +62,17 @@ struct ArrayCumSumImpl const Element x = column_const->template getValue(); const IColumn::Offsets & offsets = array.getOffsets(); - auto res_nested = ColumnVector::create(); - typename ColumnVector::Container & res_values = res_nested->getData(); + typename ColVecResult::MutablePtr res_nested; + if constexpr (IsDecimalNumber) + { + const typename ColVecType::Container & data = + checkAndGetColumn(&column_const->getDataColumn())->getData(); + res_nested = ColVecResult::create(0, data.getScale()); + } + else + res_nested = ColVecResult::create(); + + typename ColVecResult::Container & res_values = res_nested->getData(); res_values.resize(column_const->size()); size_t pos = 0; @@ -72,11 +93,16 @@ struct ArrayCumSumImpl return true; } + const typename ColVecType::Container & data = column->getData(); const IColumn::Offsets & offsets = array.getOffsets(); - const typename ColumnVector::Container & data = column->getData(); - auto res_nested = ColumnVector::create(); - typename ColumnVector::Container & res_values = res_nested->getData(); + typename ColVecResult::MutablePtr res_nested; + if constexpr (IsDecimalNumber) + res_nested = ColVecResult::create(0, data.getScale()); + else + res_nested = ColVecResult::create(); + + typename ColVecResult::Container & res_values = res_nested->getData(); res_values.resize(data.size()); size_t pos = 0; @@ -110,7 +136,10 @@ struct ArrayCumSumImpl executeType< Int32, Int64>(mapped, array, res) || executeType< Int64, Int64>(mapped, array, res) || executeType(mapped, array, res) || - executeType(mapped, array, res)) + executeType(mapped, array, res) || + executeType(mapped, array, res) || + executeType(mapped, array, res) || + executeType(mapped, array, res)) return res; else throw Exception("Unexpected column for arrayCumSum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Functions/array/arrayCumSumNonNegative.cpp b/dbms/src/Functions/array/arrayCumSumNonNegative.cpp index d27310a6b6a..4ccafaadf43 100644 --- a/dbms/src/Functions/array/arrayCumSumNonNegative.cpp +++ b/dbms/src/Functions/array/arrayCumSumNonNegative.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include "FunctionArrayMapped.h" #include @@ -34,6 +36,13 @@ struct ArrayCumSumNonNegativeImpl if (which.isFloat()) return std::make_shared(std::make_shared()); + if (which.isDecimal()) + { + UInt32 scale = getDecimalScale(*expression_return); + DataTypePtr nested = std::make_shared>(maxDecimalPrecision(), scale); + return std::make_shared(nested); + } + throw Exception("arrayCumSumNonNegativeImpl cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -41,16 +50,24 @@ struct ArrayCumSumNonNegativeImpl template static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr) { - const ColumnVector * column = checkAndGetColumn>(&*mapped); + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + + const ColVecType * column = checkAndGetColumn(&*mapped); if (!column) return false; const IColumn::Offsets & offsets = array.getOffsets(); - const typename ColumnVector::Container & data = column->getData(); + const typename ColVecType::Container & data = column->getData(); - auto res_nested = ColumnVector::create(); - typename ColumnVector::Container & res_values = res_nested->getData(); + typename ColVecResult::MutablePtr res_nested; + if constexpr (IsDecimalNumber) + res_nested = ColVecResult::create(0, data.getScale()); + else + res_nested = ColVecResult::create(); + + typename ColVecResult::Container & res_values = res_nested->getData(); res_values.resize(data.size()); size_t pos = 0; @@ -60,7 +77,7 @@ struct ArrayCumSumNonNegativeImpl // skip empty arrays if (pos < offsets[i]) { - accum_sum = data[pos] > 0 ? data[pos] : 0; + accum_sum = data[pos] > 0 ? data[pos] : Element(0); res_values[pos] = accum_sum; for (++pos; pos < offsets[i]; ++pos) { @@ -90,7 +107,10 @@ struct ArrayCumSumNonNegativeImpl executeType< Int32, Int64>(mapped, array, res) || executeType< Int64, Int64>(mapped, array, res) || executeType(mapped, array, res) || - executeType(mapped, array, res)) + executeType(mapped, array, res) || + executeType(mapped, array, res) || + executeType(mapped, array, res) || + executeType(mapped, array, res)) return res; else throw Exception("Unexpected column for arrayCumSumNonNegativeImpl: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Functions/array/arrayDifference.cpp b/dbms/src/Functions/array/arrayDifference.cpp index 4d3acb5b927..545749e5ec0 100644 --- a/dbms/src/Functions/array/arrayDifference.cpp +++ b/dbms/src/Functions/array/arrayDifference.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include "FunctionArrayMapped.h" #include @@ -37,6 +39,9 @@ struct ArrayDifferenceImpl if (which.isFloat32() || which.isFloat64()) return std::make_shared(std::make_shared()); + if (which.isDecimal()) + return std::make_shared(expression_return); + throw Exception("arrayDifference cannot process values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } @@ -44,16 +49,24 @@ struct ArrayDifferenceImpl template static bool executeType(const ColumnPtr & mapped, const ColumnArray & array, ColumnPtr & res_ptr) { - const ColumnVector * column = checkAndGetColumn>(&*mapped); + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + + const ColVecType * column = checkAndGetColumn(&*mapped); if (!column) return false; const IColumn::Offsets & offsets = array.getOffsets(); - const typename ColumnVector::Container & data = column->getData(); + const typename ColVecType::Container & data = column->getData(); - auto res_nested = ColumnVector::create(); - typename ColumnVector::Container & res_values = res_nested->getData(); + typename ColVecResult::MutablePtr res_nested; + if constexpr (IsDecimalNumber) + res_nested = ColVecResult::create(0, data.getScale()); + else + res_nested = ColVecResult::create(); + + typename ColVecResult::Container & res_values = res_nested->getData(); res_values.resize(data.size()); size_t pos = 0; @@ -87,7 +100,10 @@ struct ArrayDifferenceImpl executeType< Int32, Int64>(mapped, array, res) || executeType< Int64, Int64>(mapped, array, res) || executeType(mapped, array, res) || - executeType(mapped, array, res)) + executeType(mapped, array, res) || + executeType(mapped, array, res) || + executeType(mapped, array, res) || + executeType(mapped, array, res)) return res; else throw Exception("Unexpected column for arrayDifference: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Functions/array/arrayIntersect.cpp b/dbms/src/Functions/array/arrayIntersect.cpp index 2da0667f577..4f5adb359f7 100644 --- a/dbms/src/Functions/array/arrayIntersect.cpp +++ b/dbms/src/Functions/array/arrayIntersect.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -12,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -104,6 +106,19 @@ private: template void operator()(); }; + + struct DecimalExecutor + { + const UnpackedArrays & arrays; + const DataTypePtr & data_type; + ColumnPtr & result; + + DecimalExecutor(const UnpackedArrays & arrays_, const DataTypePtr & data_type_, ColumnPtr & result_) + : arrays(arrays_), data_type(data_type_), result(result_) {} + + template + void operator()(); + }; }; @@ -399,7 +414,8 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar ColumnPtr result_column; auto not_nullable_nested_return_type = removeNullable(nested_return_type); - TypeListNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column)); + TypeListNativeNumbers::forEach(NumberExecutor(arrays, not_nullable_nested_return_type, result_column)); + TypeListDecimalNumbers::forEach(DecimalExecutor(arrays, not_nullable_nested_return_type, result_column)); using DateMap = ClearableHashMap, HashTableGrower, @@ -445,6 +461,17 @@ void FunctionArrayIntersect::NumberExecutor::operator()() result = execute, true>(arrays, ColumnVector::create()); } +template +void FunctionArrayIntersect::DecimalExecutor::operator()() +{ + using Map = ClearableHashMap, HashTableGrower, + HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(T)>>; + + if (!result) + if (auto * decimal = typeid_cast *>(data_type.get())) + result = execute, true>(arrays, ColumnDecimal::create(0, decimal->getScale())); +} + template ColumnPtr FunctionArrayIntersect::execute(const UnpackedArrays & arrays, MutableColumnPtr result_data_ptr) { diff --git a/dbms/src/Functions/array/arraySplit.cpp b/dbms/src/Functions/array/arraySplit.cpp index 19bc599f856..c23f3b0af21 100644 --- a/dbms/src/Functions/array/arraySplit.cpp +++ b/dbms/src/Functions/array/arraySplit.cpp @@ -37,20 +37,24 @@ struct ArraySplitImpl size_t pos = 0; - out_offsets_2.reserve(in_offsets.size()); // the actual size would be equal or larger + out_offsets_2.reserve(in_offsets.size()); // assume the actual size to be equal or larger out_offsets_1.reserve(in_offsets.size()); for (size_t i = 0; i < in_offsets.size(); ++i) { - pos += !reverse; - for (; pos < in_offsets[i] - reverse; ++pos) + if (pos < in_offsets[i]) { - if (cut[pos]) - out_offsets_2.push_back(pos + reverse); - } - pos += reverse; + pos += !reverse; + for (; pos < in_offsets[i] - reverse; ++pos) + { + if (cut[pos]) + out_offsets_2.push_back(pos + reverse); + } + pos += reverse; + + out_offsets_2.push_back(pos); + } - out_offsets_2.push_back(pos); out_offsets_1.push_back(out_offsets_2.size()); } } @@ -73,13 +77,21 @@ struct ArraySplitImpl } else { + size_t pos = 0; + out_offsets_2.reserve(in_offsets.size()); out_offsets_1.reserve(in_offsets.size()); for (size_t i = 0; i < in_offsets.size(); ++i) { - out_offsets_2.push_back(in_offsets[i]); - out_offsets_1.push_back(i + 1); + if (pos < in_offsets[i]) + { + pos = in_offsets[i]; + + out_offsets_2.push_back(pos); + } + + out_offsets_1.push_back(out_offsets_2.size()); } } } diff --git a/dbms/src/Functions/array/arraySum.cpp b/dbms/src/Functions/array/arraySum.cpp index 403f7625f1d..295dec36299 100644 --- a/dbms/src/Functions/array/arraySum.cpp +++ b/dbms/src/Functions/array/arraySum.cpp @@ -1,5 +1,7 @@ #include +#include #include +#include #include "FunctionArrayMapped.h" #include @@ -31,25 +33,43 @@ struct ArraySumImpl if (which.isFloat()) return std::make_shared(); + if (which.isDecimal()) + { + UInt32 scale = getDecimalScale(*expression_return); + return std::make_shared>(maxDecimalPrecision(), scale); + } + throw Exception("arraySum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } template static bool executeType(const ColumnPtr & mapped, const ColumnArray::Offsets & offsets, ColumnPtr & res_ptr) { - const ColumnVector * column = checkAndGetColumn>(&*mapped); + using ColVecType = std::conditional_t, ColumnDecimal, ColumnVector>; + using ColVecResult = std::conditional_t, ColumnDecimal, ColumnVector>; + + const ColVecType * column = checkAndGetColumn(&*mapped); if (!column) { - const ColumnConst * column_const = checkAndGetColumnConst>(&*mapped); + const ColumnConst * column_const = checkAndGetColumnConst(&*mapped); if (!column_const) return false; const Element x = column_const->template getValue(); - auto res_column = ColumnVector::create(offsets.size()); - typename ColumnVector::Container & res = res_column->getData(); + typename ColVecResult::MutablePtr res_column; + if constexpr (IsDecimalNumber) + { + const typename ColVecType::Container & data = + checkAndGetColumn(&column_const->getDataColumn())->getData(); + res_column = ColVecResult::create(offsets.size(), data.getScale()); + } + else + res_column = ColVecResult::create(offsets.size()); + + typename ColVecResult::Container & res = res_column->getData(); size_t pos = 0; for (size_t i = 0; i < offsets.size(); ++i) @@ -62,9 +82,15 @@ struct ArraySumImpl return true; } - const typename ColumnVector::Container & data = column->getData(); - auto res_column = ColumnVector::create(offsets.size()); - typename ColumnVector::Container & res = res_column->getData(); + const typename ColVecType::Container & data = column->getData(); + + typename ColVecResult::MutablePtr res_column; + if constexpr (IsDecimalNumber) + res_column = ColVecResult::create(offsets.size(), data.getScale()); + else + res_column = ColVecResult::create(offsets.size()); + + typename ColVecResult::Container & res = res_column->getData(); size_t pos = 0; for (size_t i = 0; i < offsets.size(); ++i) @@ -95,7 +121,10 @@ struct ArraySumImpl executeType< Int32, Int64>(mapped, offsets, res) || executeType< Int64, Int64>(mapped, offsets, res) || executeType(mapped, offsets, res) || - executeType(mapped, offsets, res)) + executeType(mapped, offsets, res) || + executeType(mapped, offsets, res) || + executeType(mapped, offsets, res) || + executeType(mapped, offsets, res)) return res; else throw Exception("Unexpected column for arraySum: " + mapped->getName(), ErrorCodes::ILLEGAL_COLUMN); diff --git a/dbms/src/Functions/greatCircleDistance.cpp b/dbms/src/Functions/greatCircleDistance.cpp index 06aa60c7798..2d1c310cd40 100644 --- a/dbms/src/Functions/greatCircleDistance.cpp +++ b/dbms/src/Functions/greatCircleDistance.cpp @@ -10,9 +10,6 @@ #include #include -#define DEGREES_IN_RADIANS (M_PI / 180.0) -#define EARTH_RADIUS_IN_METERS 6372797.560856 - namespace DB { @@ -24,19 +21,109 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static inline Float64 degToRad(Float64 angle) { return angle * DEGREES_IN_RADIANS; } +namespace +{ +const double PI = 3.14159265358979323846; +const float TO_RADF = static_cast(PI / 180.0); +const float TO_RADF2 = static_cast(PI / 360.0); +const int GEODIST_TABLE_COS = 1024; // maxerr 0.00063% +const int GEODIST_TABLE_ASIN = 512; +const int GEODIST_TABLE_K = 1024; + +float g_GeoCos[GEODIST_TABLE_COS + 1]; /// cos(x) table +float g_GeoAsin[GEODIST_TABLE_ASIN + 1]; /// asin(sqrt(x)) table +float g_GeoFlatK[GEODIST_TABLE_K + 1][2]; /// geodistAdaptive() flat ellipsoid method k1,k2 coeffs table + +inline double sqr(double v) +{ + return v * v; +} + +inline float fsqr(float v) +{ + return v * v; +} + +void geodistInit() +{ + for (size_t i = 0; i <= GEODIST_TABLE_COS; ++i) + g_GeoCos[i] = static_cast(cos(2 * PI * i / GEODIST_TABLE_COS)); // [0, 2pi] -> [0, COSTABLE] + + for (size_t i = 0; i <= GEODIST_TABLE_ASIN; ++i) + g_GeoAsin[i] = static_cast(asin( + sqrt(static_cast(i) / GEODIST_TABLE_ASIN))); // [0, 1] -> [0, ASINTABLE] + + for (size_t i = 0; i <= GEODIST_TABLE_K; ++i) + { + double x = PI * i / GEODIST_TABLE_K - PI * 0.5; // [-pi/2, pi/2] -> [0, KTABLE] + g_GeoFlatK[i][0] = static_cast(sqr(111132.09 - 566.05 * cos(2 * x) + 1.20 * cos(4 * x))); + g_GeoFlatK[i][1] = static_cast(sqr(111415.13 * cos(x) - 94.55 * cos(3 * x) + 0.12 * cos(5 * x))); + } +} + +inline float geodistDegDiff(float f) +{ + f = static_cast(fabs(f)); + while (f > 360) + f -= 360; + if (f > 180) + f = 360 - f; + return f; +} + +inline float geodistFastCos(float x) +{ + float y = static_cast(fabs(x) * GEODIST_TABLE_COS / PI / 2); + int i = static_cast(y); + y -= i; + i &= (GEODIST_TABLE_COS - 1); + return g_GeoCos[i] + (g_GeoCos[i + 1] - g_GeoCos[i]) * y; +} + +inline float geodistFastSin(float x) +{ + float y = static_cast(fabs(x) * GEODIST_TABLE_COS / PI / 2); + int i = static_cast(y); + y -= i; + i = (i - GEODIST_TABLE_COS / 4) & (GEODIST_TABLE_COS - 1); // cos(x-pi/2)=sin(x), costable/4=pi/2 + return g_GeoCos[i] + (g_GeoCos[i + 1] - g_GeoCos[i]) * y; +} + + +/// fast implementation of asin(sqrt(x)) +/// max error in floats 0.00369%, in doubles 0.00072% +inline float geodistFastAsinSqrt(float x) +{ + if (x < 0.122) + { + // distance under 4546km, Taylor error under 0.00072% + float y = static_cast(sqrt(x)); + return y + x * y * 0.166666666666666f + x * x * y * 0.075f + x * x * x * y * 0.044642857142857f; + } + if (x < 0.948) + { + // distance under 17083km, 512-entry LUT error under 0.00072% + x *= GEODIST_TABLE_ASIN; + int i = static_cast(x); + return g_GeoAsin[i] + (g_GeoAsin[i + 1] - g_GeoAsin[i]) * (x - i); + } + return static_cast(asin(sqrt(x))); // distance over 17083km, just compute honestly +} +} /** * The function calculates distance in meters between two points on Earth specified by longitude and latitude in degrees. - * The function uses great circle distance formula https://en.wikipedia.org/wiki/Great-circle_distance. + * The function uses great circle distance formula https://en.wikipedia.org/wiki/Great-circle_distance . * Throws exception when one or several input values are not within reasonable bounds. - * Latitude must be in [-90, 90], longitude must be [-180, 180] - * + * Latitude must be in [-90, 90], longitude must be [-180, 180]. + * Original code of this implementation of this function is here https://github.com/sphinxsearch/sphinx/blob/409f2c2b5b2ff70b04e38f92b6b1a890326bad65/src/sphinxexpr.cpp#L3825. + * Andrey Aksenov, the author of original code, permitted to use this code in ClickHouse under the Apache 2.0 license. + * Presentation about this code from Highload++ Siberia 2019 is here https://github.com/ClickHouse/ClickHouse/files/3324740/1_._._GEODIST_._.pdf + * The main idea of this implementation is optimisations based on Taylor series, trigonometric identity and calculated constants once for cosine, arcsine(sqrt) and look up table. */ class FunctionGreatCircleDistance : public IFunction { public: - static constexpr auto name = "greatCircleDistance"; static FunctionPtr create(const Context &) { return std::make_shared(); } @@ -103,16 +190,30 @@ private: lat1Deg < -90 || lat1Deg > 90 || lat2Deg < -90 || lat2Deg > 90) { - throw Exception("Arguments values out of bounds for function " + getName(), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + throw Exception("Arguments values out of bounds for function " + getName(), + ErrorCodes::ARGUMENT_OUT_OF_BOUND); } - Float64 lon1Rad = degToRad(lon1Deg); - Float64 lat1Rad = degToRad(lat1Deg); - Float64 lon2Rad = degToRad(lon2Deg); - Float64 lat2Rad = degToRad(lat2Deg); - Float64 u = sin((lat2Rad - lat1Rad) / 2); - Float64 v = sin((lon2Rad - lon1Rad) / 2); - return 2.0 * EARTH_RADIUS_IN_METERS * asin(sqrt(u * u + cos(lat1Rad) * cos(lat2Rad) * v * v)); + float dlat = geodistDegDiff(lat1Deg - lat2Deg); + float dlon = geodistDegDiff(lon1Deg - lon2Deg); + + if (dlon < 13) + { + // points are close enough; use flat ellipsoid model + // interpolate sqr(k1), sqr(k2) coefficients using latitudes midpoint + float m = (lat1Deg + lat2Deg + 180) * GEODIST_TABLE_K / 360; // [-90, 90] degrees -> [0, KTABLE] indexes + int i = static_cast(m); + i &= (GEODIST_TABLE_K - 1); + float kk1 = g_GeoFlatK[i][0] + (g_GeoFlatK[i + 1][0] - g_GeoFlatK[i][0]) * (m - i); + float kk2 = g_GeoFlatK[i][1] + (g_GeoFlatK[i + 1][1] - g_GeoFlatK[i][1]) * (m - i); + return static_cast(sqrt(kk1 * dlat * dlat + kk2 * dlon * dlon)); + } + // points too far away; use haversine + static const float D = 2 * 6371000; + float a = fsqr(geodistFastSin(dlat * TO_RADF2)) + + geodistFastCos(lat1Deg * TO_RADF) * geodistFastCos(lat2Deg * TO_RADF) * + fsqr(geodistFastSin(dlon * TO_RADF2)); + return static_cast(D * geodistFastAsinSqrt(a)); } @@ -160,6 +261,7 @@ private: void registerFunctionGreatCircleDistance(FunctionFactory & factory) { + geodistInit(); factory.registerFunction(); } diff --git a/dbms/src/Functions/if.cpp b/dbms/src/Functions/if.cpp index f0534a13d66..aa7f924d1f9 100644 --- a/dbms/src/Functions/if.cpp +++ b/dbms/src/Functions/if.cpp @@ -175,9 +175,7 @@ public: private: template - static constexpr bool allow_arrays = - !IsDecimalNumber && !IsDecimalNumber && - !std::is_same_v && !std::is_same_v; + static constexpr bool allow_arrays = !std::is_same_v && !std::is_same_v; template static UInt32 decimalScale(Block & block [[maybe_unused]], const ColumnNumbers & arguments [[maybe_unused]]) diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 231d3bc7ff8..5f354a13b56 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -140,6 +140,7 @@ struct ContextShared ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections. InterserverIOHandler interserver_io_handler; /// Handler for interserver communication. std::optional background_pool; /// The thread pool for the background work performed by the tables. + std::optional background_move_pool; /// The thread pool for the background moves performed by the tables. std::optional schedule_pool; /// A thread pool that can run different jobs in background (used in replicated tables) MultiVersion macros; /// Substitutions extracted from config. std::unique_ptr ddl_worker; /// Process ddl commands from zk. @@ -287,6 +288,7 @@ struct ContextShared external_dictionaries_loader.reset(); external_models_loader.reset(); background_pool.reset(); + background_move_pool.reset(); schedule_pool.reset(); ddl_worker.reset(); @@ -1489,6 +1491,14 @@ BackgroundProcessingPool & Context::getBackgroundPool() return *shared->background_pool; } +BackgroundProcessingPool & Context::getBackgroundMovePool() +{ + auto lock = getLock(); + if (!shared->background_move_pool) + shared->background_move_pool.emplace(settings.background_move_pool_size, "BackgroundMovePool", "BgMoveProcPool"); + return *shared->background_move_pool; +} + BackgroundSchedulePool & Context::getSchedulePool() { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index da4566e6b2c..e6d0a7319af 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -450,6 +450,7 @@ public: void dropCaches() const; BackgroundProcessingPool & getBackgroundPool(); + BackgroundProcessingPool & getBackgroundMovePool(); BackgroundSchedulePool & getSchedulePool(); void setDDLWorker(std::unique_ptr ddl_worker); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index fc5d697d005..37239e0bd11 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -233,9 +233,16 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global) void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name) { auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); + if (prepared_sets.count(set_key)) return; /// Already prepared. + if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name)) + { + prepared_sets.insert({set_key, set_ptr_from_storage_set}); + return; + } + auto interpreter_subquery = interpretSubquery(subquery_or_table_name, context, subquery_depth + 1, {}); BlockIO res = interpreter_subquery->execute(); @@ -256,6 +263,19 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr prepared_sets[set_key] = std::move(set); } +SetPtr SelectQueryExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) +{ + const auto * table = subquery_or_table_name->as(); + if (!table) + return nullptr; + const DatabaseAndTableWithAlias database_table(*table); + const auto storage = context.getTable(database_table.database, database_table.table); + if (storage->getName() != "Set") + return nullptr; + const auto storage_set = std::dynamic_pointer_cast(storage); + return storage_set->getSet(); +} + /// Perfomance optimisation for IN() if storage supports it. void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index aac801fbd4c..8cd7b754632 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -219,6 +219,13 @@ private: */ void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); + /** + * Checks if subquery is not a plain StorageSet. + * Because while making set we will read data from StorageSet which is not allowed. + * Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise. + */ + SetPtr isPlainStorageSetInSubquery(const ASTPtr & subquery_of_table_name); + JoinPtr makeTableJoin(const ASTTablesInSelectQueryElement & join_element); void makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, NamesWithAliases && required_columns_with_aliases, SubqueryForSet & subquery_for_set) const; diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 6486b394623..7bc5aedae6d 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -975,7 +975,7 @@ private: /// do not update loadable objects with zero as lifetime const auto & lifetime = loaded_object->getLifetime(); - if (lifetime.min_sec == 0 || lifetime.max_sec == 0) + if (lifetime.min_sec == 0 && lifetime.max_sec == 0) return never; if (!error_count) @@ -1197,6 +1197,12 @@ void ExternalLoader::reload(bool load_never_loading) const loading_dispatcher->reload(load_never_loading); } +void ExternalLoader::reload(const FilterByNameFunction & filter_by_name, bool load_never_loading) const +{ + loading_dispatcher->setConfiguration(config_files_reader->read()); + loading_dispatcher->reload(filter_by_name, load_never_loading); +} + void ExternalLoader::addObjectAndLoad( const String & name, const String & external_name, diff --git a/dbms/src/Interpreters/ExternalLoader.h b/dbms/src/Interpreters/ExternalLoader.h index 1ae5efdb4b4..67be8fc5076 100644 --- a/dbms/src/Interpreters/ExternalLoader.h +++ b/dbms/src/Interpreters/ExternalLoader.h @@ -150,12 +150,16 @@ public: /// Also function can load dictionary synchronously void reload(const String & name, bool load_never_loading = false) const; - /// Starts reloading of all the objects. /// `load_never_loading` specifies what to do with the objects which have never been loading before. /// The function can either skip them (false) or load for the first time (true). void reload(bool load_never_loading = false) const; + /// Starts reloading of all objects matched `filter_by_name`. + /// `load_never_loading` specifies what to do with the objects which have never been loading before. + /// The function can either skip them (false) or load for the first time (true). + void reload(const FilterByNameFunction & filter_by_name, bool load_never_loading = false) const; + protected: virtual LoadablePtr create(const String & name, const Poco::Util::AbstractConfiguration & config, const String & key_in_config) const = 0; diff --git a/dbms/src/Processors/Formats/IInputFormat.cpp b/dbms/src/Processors/Formats/IInputFormat.cpp new file mode 100644 index 00000000000..0fbc78ea8c0 --- /dev/null +++ b/dbms/src/Processors/Formats/IInputFormat.cpp @@ -0,0 +1,30 @@ +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +IInputFormat::IInputFormat(Block header, ReadBuffer & in_) + : ISource(std::move(header)), in(in_) +{ +} + +void IInputFormat::resetParser() +{ + if (in.hasPendingData()) + throw Exception("Unread data in IInputFormat::resetParser. Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR); + + // those are protected attributes from ISource (I didn't want to propagate resetParser up there) + finished = false; + got_exception = false; + + getPort().getInputPort().reopen(); +} + +} diff --git a/dbms/src/Processors/Formats/IInputFormat.h b/dbms/src/Processors/Formats/IInputFormat.h index 424aed455a0..7fa6859da9f 100644 --- a/dbms/src/Processors/Formats/IInputFormat.h +++ b/dbms/src/Processors/Formats/IInputFormat.h @@ -23,10 +23,15 @@ protected: #pragma GCC diagnostic pop public: - IInputFormat(Block header, ReadBuffer & in_) - : ISource(std::move(header)), in(in_) - { - } + IInputFormat(Block header, ReadBuffer & in_); + + /** In some usecase (hello Kafka) we need to read a lot of tiny streams in exactly the same format. + * The recreating of parser for each small stream takes too long, so we introduce a method + * resetParser() which allow to reset the state of parser to continure reading of + * source stream w/o recreating that. + * That should be called after current buffer was fully read. + */ + virtual void resetParser(); virtual const BlockMissingValues & getMissingValues() const { diff --git a/dbms/src/Processors/Formats/IRowInputFormat.cpp b/dbms/src/Processors/Formats/IRowInputFormat.cpp index 39422bf20b5..fc9bbe146d3 100644 --- a/dbms/src/Processors/Formats/IRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/IRowInputFormat.cpp @@ -159,4 +159,13 @@ void IRowInputFormat::syncAfterError() throw Exception("Method syncAfterError is not implemented for input format", ErrorCodes::NOT_IMPLEMENTED); } +void IRowInputFormat::resetParser() +{ + IInputFormat::resetParser(); + total_rows = 0; + num_errors = 0; + block_missing_values.clear(); +} + + } diff --git a/dbms/src/Processors/Formats/IRowInputFormat.h b/dbms/src/Processors/Formats/IRowInputFormat.h index 38dcb6acc9e..436b358cb05 100644 --- a/dbms/src/Processors/Formats/IRowInputFormat.h +++ b/dbms/src/Processors/Formats/IRowInputFormat.h @@ -53,6 +53,8 @@ public: Chunk generate() override; + void resetParser() override; + protected: /** Read next row and append it to the columns. * If no more rows - return false. diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 7564e760aa6..e10d819d8a3 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -405,6 +405,14 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, bo } } +void CSVRowInputFormat::resetParser() +{ + RowInputFormatWithDiagnosticInfo::resetParser(); + column_indexes_for_input_fields.clear(); + read_columns.clear(); + have_always_default_columns = false; +} + void registerInputFormatProcessorCSV(FormatFactory & factory) { diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h index cebc8f6d6a1..9916e80a079 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -28,6 +28,7 @@ public: void readPrefix() override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; + void resetParser() override; private: bool with_names; diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp index 409b24b496f..7bd91dc170d 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.cpp @@ -256,6 +256,15 @@ void JSONEachRowRowInputFormat::syncAfterError() skipToUnescapedNextLineOrEOF(in); } +void JSONEachRowRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + nested_prefix_length = 0; + read_columns.clear(); + seen_columns.clear(); + prev_positions.clear(); +} + void registerInputFormatProcessorJSONEachRow(FormatFactory & factory) { diff --git a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index 3e7a38e593a..de999e9d158 100644 --- a/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -27,6 +27,7 @@ public: bool readRow(MutableColumns & columns, RowReadExtension & ext) override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; + void resetParser() override; private: const String & columnName(size_t i) const; diff --git a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 75ec4c0e6cd..f913087da9b 100644 --- a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -28,18 +28,16 @@ void MySQLOutputFormat::initialize() initialized = true; auto & header = getPort(PortKind::Main).getHeader(); - + data_types = header.getDataTypes(); if (header.columns()) { - packet_sender.sendPacket(LengthEncodedNumber(header.columns())); - for (const ColumnWithTypeAndName & column : header.getColumnsWithTypeAndName()) + for (size_t i = 0; i < header.columns(); i++) { - ColumnDefinition column_definition(column.name, CharacterSet::binary, 0, ColumnType::MYSQL_TYPE_STRING, - 0, 0); - packet_sender.sendPacket(column_definition); + const auto & column_name = header.getColumnsWithTypeAndName()[i].name; + packet_sender.sendPacket(getColumnDefinition(column_name, data_types[i]->getTypeId())); } if (!(context.mysql.client_capabilities & Capability::CLIENT_DEPRECATE_EOF)) @@ -52,22 +50,9 @@ void MySQLOutputFormat::initialize() void MySQLOutputFormat::consume(Chunk chunk) { - initialize(); - - auto & header = getPort(PortKind::Main).getHeader(); - - size_t rows = chunk.getNumRows(); - auto & columns = chunk.getColumns(); - - for (size_t i = 0; i < rows; i++) + for (size_t i = 0; i < chunk.getNumRows(); i++) { - ResultsetRow row_packet; - for (size_t col = 0; col < columns.size(); ++col) - { - WriteBufferFromOwnString ostr; - header.getByPosition(col).type->serializeAsText(*columns[col], i, ostr, format_settings); - row_packet.appendColumn(std::move(ostr.str())); - } + ProtocolText::ResultsetRow row_packet(data_types, chunk.getColumns(), i); packet_sender.sendPacket(row_packet); } } diff --git a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h index 39d04818dee..780a0c4ea05 100644 --- a/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h +++ b/dbms/src/Processors/Formats/Impl/MySQLOutputFormat.h @@ -37,6 +37,7 @@ private: const Context & context; MySQLProtocol::PacketSender packet_sender; FormatSettings format_settings; + DataTypes data_types; }; } diff --git a/dbms/src/Processors/Formats/Impl/NativeFormat.cpp b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp index 6ac0e30e0a8..846d1c3dbc6 100644 --- a/dbms/src/Processors/Formats/Impl/NativeFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/NativeFormat.cpp @@ -20,6 +20,15 @@ public: String getName() const override { return "NativeInputFormatFromNativeBlockInputStream"; } protected: + void resetParser() override + { + IInputFormat::resetParser(); + stream->resetParser(); + read_prefix = false; + read_suffix = false; + } + + Chunk generate() override { /// TODO: do something with totals and extremes. diff --git a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 2069c3f3cbf..c368e979495 100644 --- a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -62,6 +62,16 @@ namespace DB return res; } + void ORCBlockInputFormat::resetParser() + { + IInputFormat::resetParser(); + + file_reader.reset(); + file_data.clear(); + row_group_total = 0; + row_group_current = 0; + } + void registerInputFormatProcessorORC(FormatFactory &factory) { factory.registerInputFormatProcessor( diff --git a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h index afc65a2e912..331d60af9dd 100644 --- a/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -21,6 +21,8 @@ public: String getName() const override { return "ORCBlockInputFormat"; } + void resetParser() override; + protected: Chunk generate() override; diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp index 54bab6d7467..55c2ebccf41 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.cpp @@ -63,6 +63,17 @@ namespace DB return res; } + void ParquetBlockInputFormat::resetParser() + { + IInputFormat::resetParser(); + + file_reader.reset(); + file_data.clear(); + buffer.reset(); + row_group_total = 0; + row_group_current = 0; + } + void registerInputFormatProcessorParquet(FormatFactory &factory) { factory.registerInputFormatProcessor( diff --git a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index 172d3a365d4..94489f8fbaa 100644 --- a/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -18,6 +18,9 @@ class ParquetBlockInputFormat: public IInputFormat public: ParquetBlockInputFormat(ReadBuffer & in_, Block header_, const Context & context_); + void resetParser() override; + + String getName() const override { return "ParquetBlockInputFormat"; } protected: diff --git a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp index f4569377ea6..6fce4437219 100644 --- a/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ProtobufRowInputFormat.cpp @@ -65,7 +65,6 @@ void ProtobufRowInputFormat::syncAfterError() reader.endMessage(true); } - void registerInputFormatProcessorProtobuf(FormatFactory & factory) { factory.registerInputFormatProcessor("Protobuf", []( diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp index 0b1b7e3e44b..dcc7d64fcd2 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.cpp @@ -197,6 +197,14 @@ void TSKVRowInputFormat::syncAfterError() } +void TSKVRowInputFormat::resetParser() +{ + IRowInputFormat::resetParser(); + read_columns.clear(); + seen_columns.clear(); + name_buf.clear(); +} + void registerInputFormatProcessorTSKV(FormatFactory & factory) { factory.registerInputFormatProcessor("TSKV", []( diff --git a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.h index cd7ad69aed6..d35f2882e6d 100644 --- a/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TSKVRowInputFormat.h @@ -30,6 +30,8 @@ public: bool readRow(MutableColumns & columns, RowReadExtension &) override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; + void resetParser() override; + private: const FormatSettings format_settings; diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index edb5e26e026..d3a7b910c05 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -341,6 +341,13 @@ void TabSeparatedRowInputFormat::syncAfterError() skipToUnescapedNextLineOrEOF(in); } +void TabSeparatedRowInputFormat::resetParser() +{ + RowInputFormatWithDiagnosticInfo::resetParser(); + column_indexes_for_input_fields.clear(); + read_columns.clear(); + columns_to_fill_with_default_values.clear(); +} void registerInputFormatProcessorTabSeparated(FormatFactory & factory) { diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 9d3f0b52d11..785428bf6f0 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -26,6 +26,8 @@ public: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; + void resetParser() override; + private: bool with_names; bool with_types; diff --git a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index b77ec5417b0..d4de40eddec 100644 --- a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -496,6 +496,11 @@ void TemplateRowInputFormat::throwUnexpectedEof() ErrorCodes::CANNOT_READ_ALL_DATA); } +void TemplateRowInputFormat::resetParser() +{ + RowInputFormatWithDiagnosticInfo::resetParser(); + end_of_stream = false; +} void registerInputFormatProcessorTemplate(FormatFactory & factory) { diff --git a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h index 7b62347c37d..a9180c52606 100644 --- a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -28,6 +28,8 @@ public: bool allowSyncAfterError() const override; void syncAfterError() override; + void resetParser() override; + private: bool deserializeField(const DataTypePtr & type, IColumn & column, size_t file_column); void skipField(ColumnFormat col_format); diff --git a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 861a2320dd6..c42b638fb48 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -411,6 +411,13 @@ void ValuesBlockInputFormat::readSuffix() throw Exception("Unread data in PeekableReadBuffer will be lost. Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR); } +void ValuesBlockInputFormat::resetParser() +{ + IInputFormat::resetParser(); + // I'm not resetting parser modes here. + // There is a good chance that all messages have the same format. + total_rows = 0; +} void registerInputFormatProcessorValues(FormatFactory & factory) { diff --git a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h index ee0462f1eb4..9c2473a0a04 100644 --- a/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/ValuesBlockInputFormat.h @@ -33,6 +33,8 @@ public: String getName() const override { return "ValuesBlockInputFormat"; } + void resetParser() override; + const BlockMissingValues & getMissingValues() const override { return block_missing_values; } private: diff --git a/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp index 452cfa46acf..ae16bc47e11 100644 --- a/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp +++ b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp @@ -164,4 +164,17 @@ String RowInputFormatWithDiagnosticInfo::alignedName(const String & name, size_t return name + ", " + std::string(spaces_count, ' '); } +void RowInputFormatWithDiagnosticInfo::resetParser() +{ + IRowInputFormat::resetParser(); + row_num = 0; + bytes_read_at_start_of_buffer_on_current_row = 0; + bytes_read_at_start_of_buffer_on_prev_row = 0; + offset_of_current_row = std::numeric_limits::max(); + offset_of_prev_row = std::numeric_limits::max(); + max_length_of_column_name = 0; + max_length_of_data_type_name = 0; +} + + } diff --git a/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.h b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.h index 98dea066436..91f52f93c78 100644 --- a/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.h +++ b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.h @@ -16,6 +16,8 @@ public: String getDiagnosticInfo() override; + void resetParser() override; + protected: void updateDiagnosticInfo(); bool deserializeFieldAndPrintDiagnosticInfo(const String & col_name, const DataTypePtr & type, IColumn & column, diff --git a/dbms/src/Processors/Port.h b/dbms/src/Processors/Port.h index d37e11668e7..37d1ea9bd46 100644 --- a/dbms/src/Processors/Port.h +++ b/dbms/src/Processors/Port.h @@ -316,6 +316,17 @@ public: is_finished = true; } + void ALWAYS_INLINE reopen() + { + assumeConnected(); + + if (!isFinished()) + return; + + state->setFlags(0, State::IS_FINISHED); + is_finished = false; + } + OutputPort & getOutputPort() { assumeConnected(); diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp index 831aeaba6a0..3ce47bf9b34 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.cpp @@ -4,6 +4,8 @@ #include #include #include +#include + namespace DB { @@ -16,6 +18,7 @@ KafkaBlockInputStream::KafkaBlockInputStream( , commit_in_suffix(commit_in_suffix_) , non_virtual_header(storage.getSampleBlockNonMaterialized()) /// FIXME: add materialized columns support , virtual_header(storage.getSampleBlockForColumns({"_topic", "_key", "_offset", "_partition", "_timestamp"})) + { context.setSetting("input_format_skip_unknown_fields", 1u); // Always skip unknown fields regardless of the context (JSON or TSKV) context.setSetting("input_format_allow_errors_ratio", 0.); @@ -23,8 +26,6 @@ KafkaBlockInputStream::KafkaBlockInputStream( if (!storage.getSchemaName().empty()) context.setSetting("format_schema", storage.getSchemaName()); - - virtual_columns = virtual_header.cloneEmptyColumns(); } KafkaBlockInputStream::~KafkaBlockInputStream() @@ -62,7 +63,10 @@ Block KafkaBlockInputStream::readImpl() if (!buffer) return Block(); - auto read_callback = [this] + MutableColumns result_columns = non_virtual_header.cloneEmptyColumns(); + MutableColumns virtual_columns = virtual_header.cloneEmptyColumns(); + + auto read_callback = [&] { virtual_columns[0]->insert(buffer->currentTopic()); // "topic" virtual_columns[1]->insert(buffer->currentKey()); // "key" @@ -74,69 +78,74 @@ Block KafkaBlockInputStream::readImpl() virtual_columns[4]->insert(std::chrono::duration_cast(timestamp->get_timestamp()).count()); // "timestamp" }; - auto merge_blocks = [] (Block & block1, Block && block2) + auto input_format = FormatFactory::instance().getInputFormat( + storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size, read_callback); + + InputPort port(input_format->getPort().getHeader(), input_format.get()); + connect(input_format->getPort(), port); + port.setNeeded(); + + auto read_kafka_message = [&] { - if (!block1) + size_t new_rows = 0; + + while (true) { - // Need to make sure that resulting block has the same structure - block1 = std::move(block2); - return; + auto status = input_format->prepare(); + + switch (status) + { + case IProcessor::Status::Ready: + input_format->work(); + break; + + case IProcessor::Status::Finished: + input_format->resetParser(); + return new_rows; + + case IProcessor::Status::PortFull: + { + auto chunk = port.pull(); + new_rows = new_rows + chunk.getNumRows(); + + /// FIXME: materialize MATERIALIZED columns here. + + auto columns = chunk.detachColumns(); + for (size_t i = 0, s = columns.size(); i < s; ++i) + result_columns[i]->insertRangeFrom(*columns[i], 0, columns[i]->size()); + break; + } + case IProcessor::Status::NeedData: + case IProcessor::Status::Async: + case IProcessor::Status::Wait: + case IProcessor::Status::ExpandPipeline: + throw Exception("Source processor returned status " + IProcessor::statusToName(status), ErrorCodes::LOGICAL_ERROR); + } } - - if (!block2) - return; - - auto columns1 = block1.mutateColumns(); - auto columns2 = block2.mutateColumns(); - for (size_t i = 0, s = columns1.size(); i < s; ++i) - columns1[i]->insertRangeFrom(*columns2[i], 0, columns2[i]->size()); - block1.setColumns(std::move(columns1)); }; - auto read_kafka_message = [&, this] - { - Block result; - auto child = FormatFactory::instance().getInput( - storage.getFormatName(), *buffer, non_virtual_header, context, max_block_size, read_callback); - - while (auto block = child->read()) - { - auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); - virtual_columns = virtual_header.cloneEmptyColumns(); - - for (const auto & column : virtual_block.getColumnsWithTypeAndName()) - block.insert(column); - - /// FIXME: materialize MATERIALIZED columns here. - - merge_blocks(result, std::move(block)); - } - - return result; - }; - - Block single_block; - - UInt64 total_rows = 0; + size_t total_rows = 0; while (total_rows < max_block_size) { - auto new_block = read_kafka_message(); - auto new_rows = new_block.rows(); - total_rows += new_rows; - merge_blocks(single_block, std::move(new_block)); - + auto new_rows = read_kafka_message(); + total_rows = total_rows + new_rows; buffer->allowNext(); - if (!new_rows || !checkTimeLimit()) break; } - if (!single_block) + if (total_rows == 0) return Block(); + auto result_block = non_virtual_header.cloneWithColumns(std::move(result_columns)); + auto virtual_block = virtual_header.cloneWithColumns(std::move(virtual_columns)); + + for (const auto & column : virtual_block.getColumnsWithTypeAndName()) + result_block.insert(column); + return ConvertingBlockInputStream( context, - std::make_shared(single_block), + std::make_shared(result_block), getHeader(), ConvertingBlockInputStream::MatchColumnsMode::Name) .read(); diff --git a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h index 7aae403228b..5ab2df15ecd 100644 --- a/dbms/src/Storages/Kafka/KafkaBlockInputStream.h +++ b/dbms/src/Storages/Kafka/KafkaBlockInputStream.h @@ -33,9 +33,7 @@ private: UInt64 max_block_size; ConsumerBufferPtr buffer; - MutableColumns virtual_columns; bool broken = true, claimed = false, commit_in_suffix; - const Block non_virtual_header, virtual_header; }; diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp index a883946bc78..fa2b81a5eaa 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.cpp @@ -61,9 +61,12 @@ void BackgroundProcessingPoolTaskInfo::wake() } -BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_) +BackgroundProcessingPool::BackgroundProcessingPool(int size_, const char * log_name, const char * thread_name_) + : size(size_) + , thread_name(thread_name_) { - LOG_INFO(&Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with " << size << " threads"); + logger = &Logger::get(log_name); + LOG_INFO(logger, "Create " << log_name << " with " << size << " threads"); threads.resize(size); for (auto & thread : threads) @@ -122,7 +125,7 @@ BackgroundProcessingPool::~BackgroundProcessingPool() void BackgroundProcessingPool::threadFunction() { - setThreadName("BackgrProcPool"); + setThreadName(thread_name); { std::lock_guard lock(tasks_mutex); diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h index 748ba19032b..774db582a3e 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -46,7 +46,9 @@ public: using TaskHandle = std::shared_ptr; - BackgroundProcessingPool(int size_); + BackgroundProcessingPool(int size_, + const char * log_name = "BackgroundProcessingPool", + const char * thread_name_ = "BackgrProcPool"); size_t getNumberOfThreads() const { @@ -67,6 +69,8 @@ protected: using Threads = std::vector; const size_t size; + const char * thread_name; + Poco::Logger * logger; Tasks tasks; /// Ordered in priority. std::mutex tasks_mutex; diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 59ea60bb360..2843ff14d79 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -3473,6 +3473,11 @@ bool MergeTreeData::selectPartsAndMove() return moveParts(std::move(moving_tagger)); } +bool MergeTreeData::areBackgroundMovesNeeded() const +{ + return storage_policy->getVolumes().size() > 1; +} + bool MergeTreeData::movePartsToSpace(const DataPartsVector & parts, DiskSpace::SpacePtr space) { if (parts_mover.moves_blocker.isCancelled()) diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 464cfcea52c..27f538afc26 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -939,6 +939,8 @@ protected: /// Selects parts for move and moves them, used in background process bool selectPartsAndMove(); + bool areBackgroundMovesNeeded() const; + private: /// RAII Wrapper for atomic work with currently moving parts /// Acuire them in constructor and remove them in destructor diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index c766219a349..fce4479c16f 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -214,7 +214,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() } catch (const Coordination::Exception & e) { - LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString()); + LOG_ERROR(log, "Couldn't start replication: " << e.what() << ". " << DB::getCurrentExceptionMessage(true)); return false; } catch (const Exception & e) @@ -222,7 +222,7 @@ bool ReplicatedMergeTreeRestartingThread::tryStartup() if (e.code() != ErrorCodes::REPLICA_IS_ALREADY_ACTIVE) throw; - LOG_ERROR(log, "Couldn't start replication: " << e.what() << ", " << e.displayText() << ", stack trace:\n" << e.getStackTrace().toString()); + LOG_ERROR(log, "Couldn't start replication: " << e.what() << ". " << DB::getCurrentExceptionMessage(true)); return false; } } diff --git a/dbms/src/Storages/MergeTree/checkDataPart.cpp b/dbms/src/Storages/MergeTree/checkDataPart.cpp index 2303ec38efa..092cc78e313 100644 --- a/dbms/src/Storages/MergeTree/checkDataPart.cpp +++ b/dbms/src/Storages/MergeTree/checkDataPart.cpp @@ -217,31 +217,25 @@ MergeTreeData::DataPart::Checksums checkDataPart( MergeTreeData::DataPart::Checksums checksums_data; size_t marks_in_primary_key = 0; + if (!primary_key_data_types.empty()) { ReadBufferFromFile file_buf(path + "primary.idx"); HashingReadBuffer hashing_buf(file_buf); - if (!primary_key_data_types.empty()) - { - size_t key_size = primary_key_data_types.size(); - MutableColumns tmp_columns(key_size); + size_t key_size = primary_key_data_types.size(); + MutableColumns tmp_columns(key_size); + for (size_t j = 0; j < key_size; ++j) + tmp_columns[j] = primary_key_data_types[j]->createColumn(); + + while (!hashing_buf.eof()) + { + if (is_cancelled()) + return {}; + + ++marks_in_primary_key; for (size_t j = 0; j < key_size; ++j) - tmp_columns[j] = primary_key_data_types[j]->createColumn(); - - while (!hashing_buf.eof()) - { - if (is_cancelled()) - return {}; - - ++marks_in_primary_key; - for (size_t j = 0; j < key_size; ++j) - primary_key_data_types[j]->deserializeBinary(*tmp_columns[j].get(), hashing_buf); - } - } - else - { - hashing_buf.tryIgnore(std::numeric_limits::max()); + primary_key_data_types[j]->deserializeBinary(*tmp_columns[j].get(), hashing_buf); } size_t primary_idx_size = hashing_buf.count(); diff --git a/dbms/src/Storages/MergeTree/checkDataPart.h b/dbms/src/Storages/MergeTree/checkDataPart.h index cd7ac2b977f..936eebd17b2 100644 --- a/dbms/src/Storages/MergeTree/checkDataPart.h +++ b/dbms/src/Storages/MergeTree/checkDataPart.h @@ -15,7 +15,7 @@ namespace DB MergeTreeData::DataPart::Checksums checkDataPart( MergeTreeData::DataPartPtr data_part, bool require_checksums, - const DataTypes & primary_key_data_types, /// Check the primary key. If it is not necessary, pass an empty array. + const DataTypes & primary_key_data_types, const MergeTreeIndices & indices = {}, /// Check skip indices std::function is_cancelled = []{ return false; }); @@ -24,7 +24,7 @@ MergeTreeData::DataPart::Checksums checkDataPart( const MergeTreeIndexGranularity & index_granularity, const String & marks_file_extension, bool require_checksums, - const DataTypes & primary_key_data_types, /// Check the primary key. If it is not necessary, pass an empty array. + const DataTypes & primary_key_data_types, const MergeTreeIndices & indices = {}, /// Check skip indices std::function is_cancelled = []{ return false; }); } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 097557acad3..eec330c89da 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include @@ -254,7 +255,7 @@ BlockInputStreams StorageFile::read( const Context & context, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned /*num_streams*/) + unsigned num_streams) { const ColumnsDescription & columns_ = getColumns(); auto column_defaults = columns_.getDefaults(); @@ -268,7 +269,7 @@ BlockInputStreams StorageFile::read( std::static_pointer_cast(shared_from_this()), context, max_block_size, file_path, IStorage::chooseCompressionMethod(file_path, compression_method)); blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared(cur_block, column_defaults, context)); } - return blocks_input; + return narrowBlockInputStreams(blocks_input, num_streams); } diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 15734dbfa14..1030d2b54c5 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -17,6 +17,8 @@ #include #include #include +#include + #include #include #include @@ -196,7 +198,7 @@ BlockInputStreams StorageHDFS::read( const Context & context_, QueryProcessingStage::Enum /*processed_stage*/, size_t max_block_size, - unsigned /*num_streams*/) + unsigned num_streams) { const size_t begin_of_path = uri.find('/', uri.find("//") + 2); const String path_from_uri = uri.substr(begin_of_path); @@ -213,7 +215,7 @@ BlockInputStreams StorageHDFS::read( max_block_size, IStorage::chooseCompressionMethod(res_path, compression_method))); } - return result; + return narrowBlockInputStreams(result, num_streams); } void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &) diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index b9f5f8dfeda..d3ff30d2d95 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -99,7 +99,8 @@ void StorageMergeTree::startup() /// NOTE background task will also do the above cleanups periodically. time_after_previous_cleanup.restart(); merging_mutating_task_handle = global_context.getBackgroundPool().addTask([this] { return mergeMutateTask(); }); - moving_task_handle = global_context.getBackgroundPool().addTask([this] { return movePartsTask(); }); + if (areBackgroundMovesNeeded()) + moving_task_handle = global_context.getBackgroundMovePool().addTask([this] { return movePartsTask(); }); } @@ -115,7 +116,7 @@ void StorageMergeTree::shutdown() global_context.getBackgroundPool().removeTask(merging_mutating_task_handle); if (moving_task_handle) - global_context.getBackgroundPool().removeTask(moving_task_handle); + global_context.getBackgroundMovePool().removeTask(moving_task_handle); } diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 7f19b91ff28..76b6c40126a 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -2878,7 +2878,8 @@ void StorageReplicatedMergeTree::startup() data_parts_exchange_endpoint->getId(replica_path), data_parts_exchange_endpoint, global_context.getInterserverIOHandler()); queue_task_handle = global_context.getBackgroundPool().addTask([this] { return queueTask(); }); - move_parts_task_handle = global_context.getBackgroundPool().addTask([this] { return movePartsTask(); }); + if (areBackgroundMovesNeeded()) + move_parts_task_handle = global_context.getBackgroundMovePool().addTask([this] { return movePartsTask(); }); /// In this thread replica will be activated. restarting_thread.start(); @@ -2902,7 +2903,7 @@ void StorageReplicatedMergeTree::shutdown() queue_task_handle.reset(); if (move_parts_task_handle) - global_context.getBackgroundPool().removeTask(move_parts_task_handle); + global_context.getBackgroundMovePool().removeTask(move_parts_task_handle); move_parts_task_handle.reset(); if (data_parts_exchange_endpoint_holder) diff --git a/dbms/src/Storages/System/StorageSystemDictionaries.cpp b/dbms/src/Storages/System/StorageSystemDictionaries.cpp index c31d514cf08..73896c10e1b 100644 --- a/dbms/src/Storages/System/StorageSystemDictionaries.cpp +++ b/dbms/src/Storages/System/StorageSystemDictionaries.cpp @@ -36,10 +36,12 @@ NamesAndTypesList StorageSystemDictionaries::getNamesAndTypes() {"element_count", std::make_shared()}, {"load_factor", std::make_shared()}, {"source", std::make_shared()}, + {"lifetime_min", std::make_shared()}, + {"lifetime_max", std::make_shared()}, {"loading_start_time", std::make_shared()}, {"loading_duration", std::make_shared()}, //{ "creation_time", std::make_shared() }, - {"last_exception", std::make_shared()}, + {"last_exception", std::make_shared()} }; } @@ -77,12 +79,15 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con res_columns[i++]->insert(dict_ptr->getLoadFactor()); res_columns[i++]->insert(dict_ptr->getSource()->toString()); + const auto & lifetime = dict_ptr->getLifetime(); + res_columns[i++]->insert(lifetime.min_sec); + res_columns[i++]->insert(lifetime.max_sec); if (!last_exception) last_exception = dict_ptr->getLastException(); } else { - for (size_t j = 0; j != 10; ++j) + for (size_t j = 0; j != 12; ++j) // Number of empty fields if dict_ptr is null res_columns[i++]->insertDefault(); } @@ -93,7 +98,9 @@ void StorageSystemDictionaries::fillData(MutableColumns & res_columns, const Con res_columns[i++]->insert(getExceptionMessage(last_exception, false)); else res_columns[i++]->insertDefault(); + } } } + diff --git a/dbms/src/Storages/tests/CMakeLists.txt b/dbms/src/Storages/tests/CMakeLists.txt index c6704628620..91aaf85fe68 100644 --- a/dbms/src/Storages/tests/CMakeLists.txt +++ b/dbms/src/Storages/tests/CMakeLists.txt @@ -4,9 +4,6 @@ target_link_libraries (system_numbers PRIVATE dbms clickhouse_storages_system cl add_executable (storage_log storage_log.cpp) target_link_libraries (storage_log PRIVATE dbms) -add_executable (part_checker part_checker.cpp) -target_link_libraries (part_checker PRIVATE dbms) - add_executable (part_name part_name.cpp) target_link_libraries (part_name PRIVATE dbms) diff --git a/dbms/src/Storages/tests/part_checker.cpp b/dbms/src/Storages/tests/part_checker.cpp deleted file mode 100644 index 737192d2070..00000000000 --- a/dbms/src/Storages/tests/part_checker.cpp +++ /dev/null @@ -1,80 +0,0 @@ -#include -#include -#include -#include -#include - -using namespace DB; - -Poco::Path getMarksFile(const std::string & part_path) -{ - Poco::DirectoryIterator it(part_path); - Poco::DirectoryIterator end; - while (it != end) - { - Poco::Path p(it.path()); - auto extension = p.getExtension(); - if (extension == "mrk2" || extension == "mrk") - return p; - ++it; - } - throw Exception("Cannot find any mark file in directory " + part_path, DB::ErrorCodes::METRIKA_OTHER_ERROR); -} - -MergeTreeIndexGranularity readGranularity(const Poco::Path & mrk_file_path, size_t fixed_granularity) -{ - - MergeTreeIndexGranularity result; - auto extension = mrk_file_path.getExtension(); - - DB::ReadBufferFromFile mrk_in(mrk_file_path.toString()); - - for (size_t mark_num = 0; !mrk_in.eof(); ++mark_num) - { - UInt64 offset_in_compressed_file = 0; - UInt64 offset_in_decompressed_block = 0; - DB::readBinary(offset_in_compressed_file, mrk_in); - DB::readBinary(offset_in_decompressed_block, mrk_in); - UInt64 index_granularity_rows = 0; - if (extension == "mrk2") - DB::readBinary(index_granularity_rows, mrk_in); - else - index_granularity_rows = fixed_granularity; - result.appendMark(index_granularity_rows); - } - return result; -} - -int main(int argc, char ** argv) -{ - - Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Logger::root().setChannel(channel); - Logger::root().setLevel("trace"); - - if (argc != 4) - { - std::cerr << "Usage: " << argv[0] << " path strict index_granularity" << std::endl; - return 1; - } - - try - { - std::string full_path{argv[1]}; - - auto mrk_file_path = getMarksFile(full_path); - size_t fixed_granularity{parse(argv[3])}; - auto adaptive_granularity = readGranularity(mrk_file_path, fixed_granularity); - auto marks_file_extension = "." + mrk_file_path.getExtension(); - bool require_checksums = parse(argv[2]); - - checkDataPart(full_path, adaptive_granularity, marks_file_extension, require_checksums, {}); - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - throw; - } - - return 0; -} diff --git a/dbms/tests/integration/test_mysql_protocol/clients/golang/0.reference b/dbms/tests/integration/test_mysql_protocol/clients/golang/0.reference index a151cc2592e..5bfb8b0d1cb 100644 --- a/dbms/tests/integration/test_mysql_protocol/clients/golang/0.reference +++ b/dbms/tests/integration/test_mysql_protocol/clients/golang/0.reference @@ -1,7 +1,7 @@ Columns: a Column types: -a BINARY +a BIGINT Result: 0 1 @@ -10,7 +10,7 @@ name a Column types: name BINARY -a BINARY +a TINYINT Result: tables 1 Columns: @@ -18,6 +18,6 @@ a b Column types: a BINARY -b BINARY +b TINYINT Result: тест 1 diff --git a/dbms/tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml b/dbms/tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml index 777e2bad2e3..6e0558208e2 100644 --- a/dbms/tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml +++ b/dbms/tests/integration/test_mysql_protocol/clients/mysql/docker_compose.yml @@ -2,5 +2,7 @@ version: '2.2' services: mysql1: image: mysql:5.7 - # rewriting default command, because starting server is unnecessary - command: sleep infinity + restart: always + environment: + MYSQL_ALLOW_EMPTY_PASSWORD: 1 + command: --federated --socket /var/run/mysqld/mysqld.sock diff --git a/dbms/tests/integration/test_mysql_protocol/test.py b/dbms/tests/integration/test_mysql_protocol/test.py index f8d79cb2e32..d1ea106a70e 100644 --- a/dbms/tests/integration/test_mysql_protocol/test.py +++ b/dbms/tests/integration/test_mysql_protocol/test.py @@ -108,8 +108,52 @@ def test_mysql_client(mysql_client, server_address): assert stdout == '\n'.join(['column', '0', '0', '1', '1', '5', '5', 'tmp_column', '0', '1', '']) +def test_mysql_federated(mysql_client, server_address): + node.query('''DROP DATABASE IF EXISTS mysql_federated''', settings={"password": "123"}) + node.query('''CREATE DATABASE mysql_federated''', settings={"password": "123"}) + node.query('''CREATE TABLE mysql_federated.test (col UInt32) ENGINE = Log''', settings={"password": "123"}) + node.query('''INSERT INTO mysql_federated.test VALUES (0), (1), (5)''', settings={"password": "123"}) + + + code, (_, stderr) = mysql_client.exec_run(''' + mysql + -e "DROP SERVER IF EXISTS clickhouse;" + -e "CREATE SERVER clickhouse FOREIGN DATA WRAPPER mysql OPTIONS (USER 'default', PASSWORD '123', HOST '{host}', PORT {port}, DATABASE 'mysql_federated');" + -e "DROP DATABASE IF EXISTS mysql_federated;" + -e "CREATE DATABASE mysql_federated;" + '''.format(host=server_address, port=server_port), demux=True) + + assert code == 0 + + code, (stdout, stderr) = mysql_client.exec_run(''' + mysql + -e "CREATE TABLE mysql_federated.test(`col` int UNSIGNED) ENGINE=FEDERATED CONNECTION='clickhouse';" + -e "SELECT * FROM mysql_federated.test ORDER BY col;" + '''.format(host=server_address, port=server_port), demux=True) + + assert stdout == '\n'.join(['col', '0', '1', '5', '']) + + code, (stdout, stderr) = mysql_client.exec_run(''' + mysql + -e "INSERT INTO mysql_federated.test VALUES (0), (1), (5);" + -e "SELECT * FROM mysql_federated.test ORDER BY col;" + '''.format(host=server_address, port=server_port), demux=True) + + assert stdout == '\n'.join(['col', '0', '0', '1', '1', '5', '5', '']) + def test_python_client(server_address): + client = pymysql.connections.Connection(host=server_address, user='user_with_double_sha1', password='abacaba', database='default', port=server_port) + + with pytest.raises(pymysql.InternalError) as exc_info: + client.query('select name from tables') + + assert exc_info.value.args == (60, "Table default.tables doesn't exist.") + + cursor = client.cursor(pymysql.cursors.DictCursor) + cursor.execute("select 1 as a, 'тест' as b") + assert cursor.fetchall() == [{'a': 1, 'b': 'тест'}] + with pytest.raises(pymysql.InternalError) as exc_info: pymysql.connections.Connection(host=server_address, user='default', password='abacab', database='default', port=server_port) @@ -124,7 +168,7 @@ def test_python_client(server_address): cursor = client.cursor(pymysql.cursors.DictCursor) cursor.execute("select 1 as a, 'тест' as b") - assert cursor.fetchall() == [{'a': '1', 'b': 'тест'}] + assert cursor.fetchall() == [{'a': 1, 'b': 'тест'}] client.select_db('system') @@ -140,11 +184,14 @@ def test_python_client(server_address): cursor.execute("INSERT INTO table1 VALUES (1), (3)") cursor.execute("INSERT INTO table1 VALUES (1), (4)") cursor.execute("SELECT * FROM table1 ORDER BY a") - assert cursor.fetchall() == [{'a': '1'}, {'a': '1'}, {'a': '3'}, {'a': '4'}] + assert cursor.fetchall() == [{'a': 1}, {'a': 1}, {'a': 3}, {'a': 4}] def test_golang_client(server_address, golang_container): # type: (str, Container) -> None + with open(os.path.join(SCRIPT_DIR, 'clients', 'golang', '0.reference')) as fp: + reference = fp.read() + code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user default --password 123 --database ' 'abc'.format(host=server_address, port=server_port), demux=True) @@ -155,10 +202,12 @@ def test_golang_client(server_address, golang_container): 'default'.format(host=server_address, port=server_port), demux=True) assert code == 0 + assert stdout == reference - with open(os.path.join(SCRIPT_DIR, 'clients', 'golang', '0.reference')) as fp: - reference = fp.read() - assert stdout == reference + code, (stdout, stderr) = golang_container.exec_run('./main --host {host} --port {port} --user user_with_double_sha1 --password abacaba --database ' + 'default'.format(host=server_address, port=server_port), demux=True) + assert code == 0 + assert stdout == reference def test_php_client(server_address, php_container): @@ -171,6 +220,14 @@ def test_php_client(server_address, php_container): assert code == 0 assert stdout == 'tables\n' + code, (stdout, stderr) = php_container.exec_run('php -f test.php {host} {port} user_with_double_sha1 abacaba'.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} user_with_double_sha1 abacaba'.format(host=server_address, port=server_port), demux=True) + assert code == 0 + assert stdout == 'tables\n' + def test_mysqljs_client(server_address, nodejs_container): code, (_, stderr) = nodejs_container.exec_run('node test.js {host} {port} default 123'.format(host=server_address, port=server_port), demux=True) diff --git a/dbms/tests/performance/README.md b/dbms/tests/performance/README.md index 0a78fe481b2..ecda08a80b1 100644 --- a/dbms/tests/performance/README.md +++ b/dbms/tests/performance/README.md @@ -22,6 +22,8 @@ You can use `substitions`, `create`, `fill` and `drop` queries to prepare test. Take into account, that these tests will run in CI which consists of 56-cores and 512 RAM machines. Queries will be executed much faster than on local laptop. +If your test continued more than 10 minutes, please, add tag `long` to have an opportunity to run all tests and skip long ones. + ### How to run performance test You have to run clickhouse-server and after you can start testing: diff --git a/dbms/tests/performance/great_circle_dist.xml b/dbms/tests/performance/great_circle_dist.xml new file mode 100644 index 00000000000..aa2096f183d --- /dev/null +++ b/dbms/tests/performance/great_circle_dist.xml @@ -0,0 +1,16 @@ + + once + + + + 1000 + 10000 + + + + + SELECT count() FROM system.numbers WHERE NOT ignore(greatCircleDistance((rand() % 360) * 1. - 180, (number % 150) * 1.2 - 90, (number % 360) + toFloat64(rand()) / 4294967296 - 180, (rand() % 180) * 1. - 90)) + + SELECT count() FROM system.numbers WHERE NOT ignore(greatCircleDistance(55. + toFloat64(rand()) / 4294967296, 37. + toFloat64(rand()) / 4294967296, 55. + toFloat64(rand()) / 4294967296, 37. + toFloat64(rand()) / 4294967296)) + + diff --git a/dbms/tests/queries/0_stateless/00362_great_circle_distance.reference b/dbms/tests/queries/0_stateless/00362_great_circle_distance.reference index f3590f06943..f7b7549366e 100644 --- a/dbms/tests/queries/0_stateless/00362_great_circle_distance.reference +++ b/dbms/tests/queries/0_stateless/00362_great_circle_distance.reference @@ -1,3 +1,7 @@ -343417 -342558 0 +1 +1 +1 +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00362_great_circle_distance.sql b/dbms/tests/queries/0_stateless/00362_great_circle_distance.sql index a0fa9bb1eae..62f9e83764d 100644 --- a/dbms/tests/queries/0_stateless/00362_great_circle_distance.sql +++ b/dbms/tests/queries/0_stateless/00362_great_circle_distance.sql @@ -1,6 +1,13 @@ -SELECT floor(greatCircleDistance(33.3, 55.3, 38.7, 55.1)) AS distance; -SELECT floor(greatCircleDistance(33.3 + v, 55.3 + v, 38.7 + v , 55.1 + v)) AS distance from -( - select number + 0.1 as v from system.numbers limit 1 -); SELECT floor(greatCircleDistance(33.3, 55.3, 33.3, 55.3)) AS distance; +-- consts are from vincenty formula from geopy +-- k = '158.756175, 53.006373' +-- u = '37.531014, 55.703050' +-- y = '37.588144, 55.733842' +-- m = '37.617780, 55.755830' +-- n = '83.089598, 54.842461' +select abs(greatCircleDistance(37.531014, 55.703050, 37.588144, 55.733842) - 4964.25740448) / 4964.25740448 < 0.004; +select abs(greatCircleDistance(37.531014, 55.703050, 37.617780, 55.755830) - 8015.52288508) / 8015.52288508 < 0.004; +select abs(greatCircleDistance(37.588144, 55.733842, 37.617780, 55.755830) - 3075.27332275) / 3075.27332275 < 0.004; +select abs(greatCircleDistance(83.089598, 54.842461, 37.617780, 55.755830) - 2837839.72863) / 2837839.72863 < 0.004; +select abs(greatCircleDistance(37.617780, 55.755830, 158.756175, 53.006373) - 6802821.68814) / 6802821.68814 < 0.004; +select abs(greatCircleDistance(83.089598, 54.842461, 158.756175, 53.006373) - 4727216.39539) / 4727216.39539 < 0.004; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_array_functions.reference b/dbms/tests/queries/0_stateless/00700_decimal_array_functions.reference new file mode 100644 index 00000000000..969a8dd2f18 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00700_decimal_array_functions.reference @@ -0,0 +1,20 @@ +[0.0000,1.0000] Array(Decimal(9, 4)) +[0.00000000,1.00000000] Array(Decimal(18, 8)) +[0.00000000,1.00000000] Array(Decimal(38, 8)) +- +1.0000 Decimal(38, 4) +1.00000000 Decimal(38, 8) +1.00000000 Decimal(38, 8) +- +[1.0000,2.0000] Array(Decimal(38, 4)) +[1.00000000,2.00000000] Array(Decimal(38, 8)) +[1.00000000,2.00000000] Array(Decimal(38, 8)) +- +[1.0000,2.0000] Array(Decimal(38, 4)) +[1.00000000,2.00000000] Array(Decimal(38, 8)) +[1.00000000,2.00000000] Array(Decimal(38, 8)) +- +[1.0000] Array(Decimal(9, 4)) +[1.00000000] Array(Decimal(18, 8)) +[1.00000000] Array(Decimal(38, 8)) +- diff --git a/dbms/tests/queries/0_stateless/00700_decimal_array_functions.sql b/dbms/tests/queries/0_stateless/00700_decimal_array_functions.sql new file mode 100644 index 00000000000..c76c8728e15 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00700_decimal_array_functions.sql @@ -0,0 +1,20 @@ +SELECT arrayDifference([toDecimal32(0.0,4), toDecimal32(1.0,4)]) x, toTypeName(x); +SELECT arrayDifference([toDecimal64(0.0,8), toDecimal64(1.0,8)]) x, toTypeName(x); +SELECT arrayDifference([toDecimal128(0.0,8), toDecimal128(1.0,8)]) x, toTypeName(x); +SELECT '-'; +SELECT arraySum([toDecimal32(0.0,4), toDecimal32(1.0,4)]) x, toTypeName(x); +SELECT arraySum([toDecimal64(0.0,8), toDecimal64(1.0,8)]) x, toTypeName(x); +SELECT arraySum([toDecimal128(0.0,8), toDecimal128(1.0,8)]) x, toTypeName(x); +SELECT '-'; +SELECT arrayCumSum([toDecimal32(1.0,4), toDecimal32(1.0,4)]) x, toTypeName(x); +SELECT arrayCumSum([toDecimal64(1.0,8), toDecimal64(1.0,8)]) x, toTypeName(x); +SELECT arrayCumSum([toDecimal128(1.0,8), toDecimal128(1.0,8)]) x, toTypeName(x); +SELECT '-'; +SELECT arrayCumSumNonNegative([toDecimal32(1.0,4), toDecimal32(1.0,4)]) x, toTypeName(x); +SELECT arrayCumSumNonNegative([toDecimal64(1.0,8), toDecimal64(1.0,8)]) x, toTypeName(x); +SELECT arrayCumSumNonNegative([toDecimal128(1.0,8), toDecimal128(1.0,8)]) x, toTypeName(x); +SELECT '-'; +SELECT arrayCompact([toDecimal32(1.0,4), toDecimal32(1.0,4)]) x, toTypeName(x); +SELECT arrayCompact([toDecimal64(1.0,8), toDecimal64(1.0,8)]) x, toTypeName(x); +SELECT arrayCompact([toDecimal128(1.0,8), toDecimal128(1.0,8)]) x, toTypeName(x); +SELECT '-'; diff --git a/dbms/tests/queries/0_stateless/00700_decimal_gathers.reference b/dbms/tests/queries/0_stateless/00700_decimal_gathers.reference new file mode 100644 index 00000000000..bbfd7388e12 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00700_decimal_gathers.reference @@ -0,0 +1,13 @@ +[2.000] +[2.0000000000] +[2.000000000000000000] +[1.000] +[1.0000000000] +[1.000000000000000000] +- +[2.000] +[1] +[2.000000000000000000] +[1.000] +[2] +[1.000000000000000000] diff --git a/dbms/tests/queries/0_stateless/00700_decimal_gathers.sql b/dbms/tests/queries/0_stateless/00700_decimal_gathers.sql new file mode 100644 index 00000000000..98519577b62 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00700_decimal_gathers.sql @@ -0,0 +1,17 @@ +select if(1, [cast(materialize(2.0),'Decimal(9,3)')], [cast(materialize(1.0),'Decimal(9,3)')]); +select if(1, [cast(materialize(2.0),'Decimal(18,10)')], [cast(materialize(1.0),'Decimal(18,10)')]); +select if(1, [cast(materialize(2.0),'Decimal(38,18)')], [cast(materialize(1.0),'Decimal(38,18)')]); + +select if(0, [cast(materialize(2.0),'Decimal(9,3)')], [cast(materialize(1.0),'Decimal(9,3)')]); +select if(0, [cast(materialize(2.0),'Decimal(18,10)')], [cast(materialize(1.0),'Decimal(18,10)')]); +select if(0, [cast(materialize(2.0),'Decimal(38,18)')], [cast(materialize(1.0),'Decimal(38,18)')]); + +select '-'; + +select if(1, [cast(materialize(2.0),'Decimal(9,3)')], [cast(materialize(1.0),'Decimal(9,0)')]); +select if(0, [cast(materialize(2.0),'Decimal(18,10)')], [cast(materialize(1.0),'Decimal(18,0)')]); +select if(1, [cast(materialize(2.0),'Decimal(38,18)')], [cast(materialize(1.0),'Decimal(38,8)')]); + +select if(0, [cast(materialize(2.0),'Decimal(9,0)')], [cast(materialize(1.0),'Decimal(9,3)')]); +select if(1, [cast(materialize(2.0),'Decimal(18,0)')], [cast(materialize(1.0),'Decimal(18,10)')]); +select if(0, [cast(materialize(2.0),'Decimal(38,0)')], [cast(materialize(1.0),'Decimal(38,18)')]); diff --git a/dbms/tests/queries/0_stateless/01015_array_split.reference b/dbms/tests/queries/0_stateless/01015_array_split.reference index ea9d36a95b2..652e7ccc43c 100644 --- a/dbms/tests/queries/0_stateless/01015_array_split.reference +++ b/dbms/tests/queries/0_stateless/01015_array_split.reference @@ -6,8 +6,10 @@ [[1],[2],[3],[4],[5]] [[1,2],[3,4],[5]] [[1],[2,3],[4,5]] -[[]] -[[]] +[] +[] +[] +[] [] [] [[1]] diff --git a/dbms/tests/queries/0_stateless/01015_array_split.sql b/dbms/tests/queries/0_stateless/01015_array_split.sql index 64d456ed724..8ae96ba01e6 100644 --- a/dbms/tests/queries/0_stateless/01015_array_split.sql +++ b/dbms/tests/queries/0_stateless/01015_array_split.sql @@ -12,6 +12,8 @@ SELECT arraySplit(x -> 0, []); SELECT arrayReverseSplit(x -> 0, []); SELECT arraySplit(x -> 1, []); SELECT arrayReverseSplit(x -> 1, []); +SELECT arraySplit(x -> x, emptyArrayUInt8()); +SELECT arrayReverseSplit(x -> x, emptyArrayUInt8()); SELECT arraySplit(x -> x % 2 = 1, [1]); SELECT arrayReverseSplit(x -> x % 2 = 1, [1]); diff --git a/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.reference b/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.reference new file mode 100644 index 00000000000..3c6d3acf6f4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.reference @@ -0,0 +1,11 @@ +1 +2 +3 +1 +2 +3 +4 +5 +1 +2 +3 diff --git a/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.sql b/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.sql new file mode 100644 index 00000000000..8b406cae769 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01030_storage_set_supports_read.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS userid_test; + +SET use_index_for_in_with_subqueries = 1; + +CREATE TABLE userid_test (userid UInt64) ENGINE = MergeTree() PARTITION BY (intDiv(userid, 500)) ORDER BY (userid) SETTINGS index_granularity = 8192; + +INSERT INTO userid_test VALUES (1),(2),(3),(4),(5); + +DROP TABLE IF EXISTS userid_set; + +CREATE TABLE userid_set(userid UInt64) ENGINE = Set; + +INSERT INTO userid_set VALUES (1),(2),(3); + +SELECT * FROM userid_test WHERE userid IN (1, 2, 3); + +SELECT * FROM userid_test WHERE toUInt64(1) IN (userid_set); + +SELECT * FROM userid_test WHERE userid IN (userid_set); + +DROP TABLE userid_test; diff --git a/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.reference b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.reference new file mode 100644 index 00000000000..b69b141bbe4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.reference @@ -0,0 +1,2 @@ +INITIALIZING DICTIONARY +1 10 diff --git a/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql new file mode 100644 index 00000000000..0497349f86f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql @@ -0,0 +1,48 @@ +SET send_logs_level = 'none'; + +DROP DATABASE IF EXISTS database_for_dict; + +CREATE DATABASE database_for_dict Engine = Ordinary; + +DROP TABLE IF EXISTS database_for_dict.table_for_dict; + +CREATE TABLE database_for_dict.table_for_dict +( + key_column UInt64, + second_column UInt8, + third_column String +) +ENGINE = MergeTree() +ORDER BY key_column; + +INSERT INTO database_for_dict.table_for_dict VALUES (1, 100, 'Hello world'); + +DROP DATABASE IF EXISTS ordinary_db; + +CREATE DATABASE ordinary_db ENGINE = Ordinary; + +DROP DICTIONARY IF EXISTS ordinary_db.dict1; + +CREATE DICTIONARY ordinary_db.dict1 +( + key_column UInt64 DEFAULT 0, + second_column UInt8 DEFAULT 1, + third_column String DEFAULT 'qqq' +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' PASSWORD '' DB 'database_for_dict')) +LIFETIME(MIN 1 MAX 10) +LAYOUT(FLAT()); + +SELECT 'INITIALIZING DICTIONARY'; + +SELECT lifetime_min, lifetime_max FROM system.dictionaries WHERE name = 'dict1'; + +DROP DICTIONARY IF EXISTS ordinary_db.dict1; + +DROP DATABASE IF EXISTS ordinary_db; + +DROP TABLE IF EXISTS database_for_dict.table_for_dict; + +DROP DATABASE IF EXISTS database_for_dict; + diff --git a/dbms/tests/queries/0_stateless/01035_lc_empty_part_bug.reference b/dbms/tests/queries/0_stateless/01035_lc_empty_part_bug.reference new file mode 100644 index 00000000000..1ca0ea26354 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01035_lc_empty_part_bug.reference @@ -0,0 +1,3 @@ +Waiting for mutation to finish +still alive +100 diff --git a/dbms/tests/queries/0_stateless/01035_lc_empty_part_bug.sh b/dbms/tests/queries/0_stateless/01035_lc_empty_part_bug.sh new file mode 100755 index 00000000000..21f029f27f1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01035_lc_empty_part_bug.sh @@ -0,0 +1,26 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh +. $CURDIR/mergetree_mutations.lib + +# that test is failing on versions <= 19.11.12 + +${CLICKHOUSE_CLIENT} --multiquery --query=" + DROP TABLE IF EXISTS lc_empty_part_bug; + create table lc_empty_part_bug (id UInt64, s String) Engine=MergeTree ORDER BY id; + insert into lc_empty_part_bug select number as id, toString(rand()) from numbers(100); + alter table lc_empty_part_bug delete where id < 100; +" + +wait_for_mutation 'lc_empty_part_bug' 'mutation_2.txt' + +echo 'Waiting for mutation to finish' + +${CLICKHOUSE_CLIENT} --multiquery --query=" + alter table lc_empty_part_bug modify column s LowCardinality(String); + SELECT 'still alive'; + insert into lc_empty_part_bug select number+100 as id, toString(rand()) from numbers(100); + SELECT count() FROM lc_empty_part_bug WHERE not ignore(*); + DROP TABLE lc_empty_part_bug; +" diff --git a/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.reference b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.reference new file mode 100644 index 00000000000..e7190712871 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.reference @@ -0,0 +1,9 @@ +0 +10 +1 +SYSTEM RELOAD DICTIONARY +0 +10 +1 +CREATE DATABASE +1 diff --git a/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql new file mode 100644 index 00000000000..93bb56264ee --- /dev/null +++ b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database.sql @@ -0,0 +1,28 @@ +DROP DATABASE IF EXISTS dict_db_01036; +CREATE DATABASE dict_db_01036; + +CREATE TABLE dict_db_01036.dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY dict_db_01036.dict +( + key UInt64 DEFAULT 0, + val UInt64 DEFAULT 10 +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '' DB 'dict_db_01036')) +LIFETIME(MIN 0 MAX 0) +LAYOUT(FLAT()); + +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; + +SELECT 'SYSTEM RELOAD DICTIONARY'; +SYSTEM RELOAD DICTIONARY 'dict_db_01036.dict'; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; +SELECT dictGetUInt64('dict_db_01036.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; + +SELECT 'CREATE DATABASE'; +DROP DATABASE IF EXISTS empty_db_01036; +CREATE DATABASE empty_db_01036; +SELECT query_count FROM system.dictionaries WHERE database = 'dict_db_01036' AND name = 'dict'; diff --git a/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database_2.reference b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database_2.reference new file mode 100644 index 00000000000..e7190712871 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database_2.reference @@ -0,0 +1,9 @@ +0 +10 +1 +SYSTEM RELOAD DICTIONARY +0 +10 +1 +CREATE DATABASE +1 diff --git a/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database_2.sql b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database_2.sql new file mode 100644 index 00000000000..75e77467a89 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01036_no_superfluous_dict_reload_on_create_database_2.sql @@ -0,0 +1,28 @@ +DROP DATABASE IF EXISTS `foo 1234`; +CREATE DATABASE `foo 1234`; + +CREATE TABLE `foo 1234`.dict_data (key UInt64, val UInt64) Engine=Memory(); +CREATE DICTIONARY `foo 1234`.dict +( + key UInt64 DEFAULT 0, + val UInt64 DEFAULT 10 +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_data' PASSWORD '' DB 'foo 1234')) +LIFETIME(MIN 0 MAX 0) +LAYOUT(FLAT()); + +SELECT query_count FROM system.dictionaries WHERE database = 'foo 1234' AND name = 'dict'; +SELECT dictGetUInt64('foo 1234.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'foo 1234' AND name = 'dict'; + +SELECT 'SYSTEM RELOAD DICTIONARY'; +SYSTEM RELOAD DICTIONARY 'foo 1234.dict'; +SELECT query_count FROM system.dictionaries WHERE database = 'foo 1234' AND name = 'dict'; +SELECT dictGetUInt64('foo 1234.dict', 'val', toUInt64(0)); +SELECT query_count FROM system.dictionaries WHERE database = 'foo 1234' AND name = 'dict'; + +SELECT 'CREATE DATABASE'; +DROP DATABASE IF EXISTS `foo 123`; +CREATE DATABASE `foo 123`; +SELECT query_count FROM system.dictionaries WHERE database = 'foo 1234' AND name = 'dict'; diff --git a/dbms/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.reference b/dbms/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.reference new file mode 100644 index 00000000000..c47539e2301 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.reference @@ -0,0 +1,2 @@ +all_1_1_0 1 +all_0_0_0 1 diff --git a/dbms/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql b/dbms/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql new file mode 100644 index 00000000000..1d195b0388f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01037_zookeeper_check_table_empty_pk.sql @@ -0,0 +1,22 @@ +SET check_query_single_value_result = 0; +SET send_logs_level = 'none'; + +DROP TABLE IF EXISTS mt_without_pk; + +CREATE TABLE mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = MergeTree() ORDER BY tuple(); + +INSERT INTO mt_without_pk VALUES (1, 2); + +CHECK TABLE mt_without_pk; + +DROP TABLE IF EXISTS mt_without_pk; + +DROP TABLE IF EXISTS replicated_mt_without_pk; + +CREATE TABLE replicated_mt_without_pk (SomeField1 Int64, SomeField2 Double) ENGINE = ReplicatedMergeTree('/clickhouse/tables/replicated_mt_without_pk', '1') ORDER BY tuple(); + +INSERT INTO replicated_mt_without_pk VALUES (1, 2); + +CHECK TABLE replicated_mt_without_pk; + +DROP TABLE IF EXISTS replicated_mt_without_pk; diff --git a/dbms/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.reference b/dbms/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.reference new file mode 100644 index 00000000000..a2fff10e1ab --- /dev/null +++ b/dbms/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.reference @@ -0,0 +1,4 @@ +1.1 +77.77 +1.1 +2.2 diff --git a/dbms/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh b/dbms/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh new file mode 100755 index 00000000000..d3153be5e68 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01038_dictionary_lifetime_min_zero_sec.sh @@ -0,0 +1,60 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS database_for_dict" + +$CLICKHOUSE_CLIENT --query "CREATE DATABASE database_for_dict Engine = Ordinary" + + +$CLICKHOUSE_CLIENT --query " +CREATE TABLE database_for_dict.table_for_dict +( + key_column UInt64, + value Float64 +) +ENGINE = MergeTree() +ORDER BY key_column" + +$CLICKHOUSE_CLIENT --query "INSERT INTO database_for_dict.table_for_dict VALUES (1, 1.1)" + +$CLICKHOUSE_CLIENT --query " +CREATE DICTIONARY database_for_dict.dict_with_zero_min_lifetime +( + key_column UInt64, + value Float64 DEFAULT 77.77 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'database_for_dict')) +LIFETIME(1) +LAYOUT(FLAT())" + +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(1))" + +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))" + +$CLICKHOUSE_CLIENT --query "INSERT INTO database_for_dict.table_for_dict VALUES (2, 2.2)" + + +function check() +{ + + query_result=`$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))"` + + while [ $query_result != "2.2" ] + do + query_result=`$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))"` + done +} + + +export -f check; + +timeout 10 bash -c check + +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(1))" + +$CLICKHOUSE_CLIENT --query "SELECT dictGetFloat64('database_for_dict.dict_with_zero_min_lifetime', 'value', toUInt64(2))" + +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS database_for_dict" diff --git a/docs/en/development/build_cross_arm.md b/docs/en/development/build_cross_arm.md new file mode 100644 index 00000000000..4474c72c3f0 --- /dev/null +++ b/docs/en/development/build_cross_arm.md @@ -0,0 +1,35 @@ +# How to Build ClickHouse on Linux for AARCH64 (ARM64) architecture + +This is for the case when you have Linux machine and want to use it to build `clickhouse` binary that will run on another Linux machine with AARCH64 CPU architecture. This is intended for continuous integration checks that run on Linux servers. + +The cross-build for AARCH64 is based on the [Build instructions](build.md), follow them first. + +# Install Clang-8 + +Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup. +For example, in Ubuntu Bionic you can use the following commands: + +```bash +sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list +sudo apt-get install clang-8 +``` + +# Install Cross-Compilation Toolset + +```bash +cd ClickHouse +cd cmake/toolchain/linux-aarch64 +wget https://developer.arm.com/-/media/Files/downloads/gnu-a/8.3-2019.03/binrel/gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz?revision=2e88a73f-d233-4f96-b1f4-d8b36e9bb0b9&la=en -O gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz +tar --strip-components=1 xJf gcc-arm-8.3-2019.03-x86_64-aarch64-linux-gnu.tar.xz +``` + +# Build ClickHouse + +```bash +cd ClickHouse +mkdir build-arm64 +CC=clang-8 CXX=clang++-8 cmake . -Bbuild-arm64 -DCMAKE_TOOLCHAIN_FILE=cmake/linux/toolchain-aarch64.cmake +ninja -C build-arm64 +``` + +The resulting binary will run only on Linux with the AARCH64 CPU architecture. diff --git a/docs/en/development/build_cross.md b/docs/en/development/build_cross_osx.md similarity index 78% rename from docs/en/development/build_cross.md rename to docs/en/development/build_cross_osx.md index 61f0acf5b76..d204620f2a8 100644 --- a/docs/en/development/build_cross.md +++ b/docs/en/development/build_cross_osx.md @@ -2,7 +2,7 @@ This is for the case when you have Linux machine and want to use it to build `clickhouse` binary that will run on OS X. This is intended for continuous integration checks that run on Linux servers. If you want to build ClickHouse directly on Mac OS X, then proceed with another instruction: https://clickhouse.yandex/docs/en/development/build_osx/ -The cross-build for Mac OS X is based on the Build instructions, follow them first. +The cross-build for Mac OS X is based on the [Build instructions](build.md), follow them first. # Install Clang-8 @@ -31,10 +31,15 @@ git clone https://github.com/tpoechtrager/cctools-port.git cd cctools-port/cctools ./configure --prefix=${CCTOOLS} --with-libtapi=${CCTOOLS} --target=x86_64-apple-darwin make install +``` -cd ${CCTOOLS} +Also, we need to download MacOS X SDK into the working tree. + +```bash +cd ClickHouse +cd cmake/toolchain/darwin-x86_64 wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz -tar xJf MacOSX10.14.sdk.tar.xz +tar --strip-components=1 xJf MacOSX10.14.sdk.tar.xz ``` # Build ClickHouse @@ -42,11 +47,10 @@ tar xJf MacOSX10.14.sdk.tar.xz ```bash cd ClickHouse mkdir build-osx -CC=clang-8 CXX=clang++-8 cmake . -Bbuild-osx -DCMAKE_SYSTEM_NAME=Darwin \ +CC=clang-8 CXX=clang++-8 cmake . -Bbuild-osx -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake \ -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar \ -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib \ - -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld \ - -DSDK_PATH=${CCTOOLS}/MacOSX10.14.sdk + -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld ninja -C build-osx ``` diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md index 394313fccdb..198e5f5c094 100644 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -1,8 +1,8 @@ # Command-line Client -ClickHouse provides the native command line client `clickhouse-client`. The client supports command-line options and configuration files. For more information, see "[Configuring](#interfaces_cli_configuration)". +ClickHouse provides a native command-line client: `clickhouse-client`. The client supports command-line options and configuration files. For more information, see [Configuring](#interfaces_cli_configuration). -[Install](../getting_started/index.md) it by the `clickhouse-client` package and run it by the command `clickhouse-client`. +[Install](../getting_started/index.md) it from the `clickhouse-client` package and run it with the command `clickhouse-client`. ```bash $ clickhouse-client @@ -13,7 +13,7 @@ Connected to ClickHouse server version 19.17.1 revision 54428. :) ``` -Different versions of client and server are compatible, but some features may be disabled for older clients. We don't recommend using different versions of the client and the server app. When you try to use the client of the older version, then the server, `clickhouse-client` displays the message: +Different client and server versions are compatible with one another, but some features may not be available in older clients. We recommend using the same version of the client as the server app. When you try to use a client of the older version, then the server, `clickhouse-client` displays the message: ``` ClickHouse client version is older than ClickHouse server. It may lack support for new features. @@ -45,9 +45,9 @@ Similarly, to process a large number of queries, you can run 'clickhouse-client' In interactive mode, you get a command line where you can enter queries. -If 'multiline' is not specified (the default):To run the query, press Enter. The semicolon is not necessary at the end of the query. To enter a multiline query, enter a backslash `\` before the line feed. After you press Enter, you will be asked to enter the next line of the query. +If 'multiline' is not specified (the default): To run the query, press Enter. The semicolon is not necessary at the end of the query. To enter a multiline query, enter a backslash `\` before the line feed. After you press Enter, you will be asked to enter the next line of the query. -If multiline is specified:To run a query, end it with a semicolon and press Enter. If the semicolon was omitted at the end of the entered line, you will be asked to enter the next line of the query. +If multiline is specified: To run a query, end it with a semicolon and press Enter. If the semicolon was omitted at the end of the entered line, you will be asked to enter the next line of the query. Only a single query is run, so everything after the semicolon is ignored. diff --git a/docs/en/interfaces/http.md b/docs/en/interfaces/http.md index ee05a1cdb64..25a146f78b3 100644 --- a/docs/en/interfaces/http.md +++ b/docs/en/interfaces/http.md @@ -28,8 +28,12 @@ $ wget -O- -q 'http://localhost:8123/?query=SELECT 1' $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 HTTP/1.0 200 OK +Date: Wed, 27 Nov 2019 10:30:18 GMT Connection: Close -Date: Fri, 16 Nov 2012 19:21:50 GMT +Content-Type: text/tab-separated-values; charset=UTF-8 +X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal +X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} 1 ``` diff --git a/docs/en/operations/monitoring.md b/docs/en/operations/monitoring.md index eaa0ffdd406..331c3c0144f 100644 --- a/docs/en/operations/monitoring.md +++ b/docs/en/operations/monitoring.md @@ -34,4 +34,4 @@ You can configure ClickHouse to export metrics to [Graphite](https://github.com/ Additionally, you can monitor server availability through the HTTP API. Send the `HTTP GET` request to `/`. If the server is available, it responds with `200 OK`. -To monitor servers in a cluster configuration, you should set the [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) parameter and use the HTTP resource `/replicas-delay`. A request to `/replicas-delay` returns `200 OK` if the replica is available and is not delayed behind the other replicas. If a replica is delayed, it returns information about the gap. +To monitor servers in a cluster configuration, you should set the [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) parameter and use the HTTP resource `/replicas_status`. A request to `/replicas_status` returns `200 OK` if the replica is available and is not delayed behind the other replicas. If a replica is delayed, it returns information about the gap. diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 4158cad3440..c76637cc927 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -578,6 +578,32 @@ If the table doesn't exist, ClickHouse will create it. If the structure of the q ``` +## query_thread_log {#server_settings-query-thread-log} + +Setting for logging threads of queries received with the [log_query_threads=1](../settings/settings.md#settings-log-query-threads) setting. + +Queries are logged in the [system.query_thread_log](../system_tables.md#system_tables-query-thread-log) table, not in a separate file. You can change the name of the table in the `table` parameter (see below). + +Use the following parameters to configure logging: + +- `database` – Name of the database. +- `table` – Name of the system table the queries will be logged in. +- `partition_by` – Sets a [custom partitioning key](../../operations/table_engines/custom_partitioning_key.md) for a system table. +- `flush_interval_milliseconds` – Interval for flushing data from the buffer in memory to the table. + +If the table doesn't exist, ClickHouse will create it. If the structure of the query thread log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. + +**Example** + +```xml + + system + query_thread_log
+ toMonday(event_date) + 7500 +
+``` + ## trace_log {#server_settings-trace_log} Settings for the [trace_log](../system_tables.md#system_tables-trace_log) system table operation. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0ad80cfed2f..7928f2bfe5b 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -513,6 +513,16 @@ Queries sent to ClickHouse with this setup are logged according to the rules in log_queries=1 +## log_query_threads {#settings-log-query-threads} + +Setting up query threads logging. + +Queries' threads runned by ClickHouse with this setup are logged according to the rules in the [query_thread_log](../server_settings/settings.md#server_settings-query-thread-log) server configuration parameter. + +**Example**: + + log_query_threads=1 + ## max_insert_block_size {#settings-max_insert_block_size} The size of blocks to form for insertion into a table. @@ -595,6 +605,13 @@ Timeouts in seconds on the socket used for communicating with the client. Default value: 10, 300, 300. +## cancel_http_readonly_queries_on_client_close + +Cancels HTTP readonly queries (e.g. SELECT) when a client closes the connection without waiting for response. + +Default value: 0 + + ## poll_interval Lock in a wait loop for the specified number of seconds. diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index e85d5225763..373b87fbf17 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -206,7 +206,7 @@ Columns: ## system.graphite_retentions -Contains information about parameters [graphite_rollup](server_settings/settings.md#server_settings-graphite_rollup) which are used in tables with [*GraphiteMergeTree](table_engines/graphitemergetree.md) engines. +Contains information about parameters [graphite_rollup](server_settings/settings.md#server_settings-graphite_rollup) which are used in tables with [\*GraphiteMergeTree](table_engines/graphitemergetree.md) engines. Columns: @@ -418,8 +418,8 @@ Columns: - `'QueryFinish' = 2` — Successful end of query execution. - `'ExceptionBeforeStart' = 3` — Exception before the start of query execution. - `'ExceptionWhileProcessing' = 4` — Exception during the query execution. -- `event_date` (Date) — Event date. -- `event_time` (DateTime) — Event time. +- `event_date` (Date) — Query starting date. +- `event_time` (DateTime) — Query starting time. - `query_start_time` (DateTime) — Start time of query execution. - `query_duration_ms` (UInt64) — Duration of query execution. - `read_rows` (UInt64) — Number of read rows. @@ -437,36 +437,32 @@ Columns: - 0 — Query was initiated by another query for distributed query execution. - `user` (String) — Name of the user who initiated the current query. - `query_id` (String) — ID of the query. -- `address` (FixedString(16)) — IP address the query was initiated from. -- `port` (UInt16) — The server port that was used to receive the query. -- `initial_user` (String) — Name of the user who ran the parent query (for distributed query execution). -- `initial_query_id` (String) — ID of the parent query. -- `initial_address` (FixedString(16)) — IP address that the parent query was launched from. -- `initial_port` (UInt16) — The server port that was used to receive the parent query from the client. +- `address` (IPv6) — IP address that was used to make the query. +- `port` (UInt16) — The client port that was used to make the query. +- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). +- `initial_query_id` (String) — ID of the initial query (for distributed query execution). +- `initial_address` (IPv6) — IP address that the parent query was launched from. +- `initial_port` (UInt16) — The client port that was used to make the parent query. - `interface` (UInt8) — Interface that the query was initiated from. Possible values: - 1 — TCP. - 2 — HTTP. -- `os_user` (String) — User's OS. -- `client_hostname` (String) — Server name that the [clickhouse-client](../interfaces/cli.md) is connected to. -- `client_name` (String) — The [clickhouse-client](../interfaces/cli.md) name. -- `client_revision` (UInt32) — Revision of the [clickhouse-client](../interfaces/cli.md). -- `client_version_major` (UInt32) — Major version of the [clickhouse-client](../interfaces/cli.md). -- `client_version_minor` (UInt32) — Minor version of the [clickhouse-client](../interfaces/cli.md). -- `client_version_patch` (UInt32) — Patch component of the [clickhouse-client](../interfaces/cli.md) version. +- `os_user` (String) — OS's username who runs [clickhouse-client](../interfaces/cli.md). +- `client_hostname` (String) — Hostname of the client machine where the [clickhouse-client](../interfaces/cli.md) or another TCP client is run. +- `client_name` (String) — The [clickhouse-client](../interfaces/cli.md) or another TCP client name. +- `client_revision` (UInt32) — Revision of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_major` (UInt32) — Major version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_minor` (UInt32) — Minor version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_patch` (UInt32) — Patch component of the [clickhouse-client](../interfaces/cli.md) or another TCP client version. - `http_method` (UInt8) — HTTP method that initiated the query. Possible values: - 0 — The query was launched from the TCP interface. - 1 — `GET` method was used. - 2 — `POST` method was used. - `http_user_agent` (String) — The `UserAgent` header passed in the HTTP request. -- `quota_key` (String) — The quota key specified in the [quotas](quotas.md) setting. +- `quota_key` (String) — The "quota key" specified in the [quotas](quotas.md) setting (see `keyed`). - `revision` (UInt32) — ClickHouse revision. - `thread_numbers` (Array(UInt32)) — Number of threads that are participating in query execution. -- `ProfileEvents.Names` (Array(String)) — Counters that measure the following metrics: - - Time spent on reading and writing over the network. - - Time spent on reading and writing to a disk. - - Number of network errors. - - Time spent on waiting when the network bandwidth is limited. -- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` column. +- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics. The description of them could be found in the table [system.events](#system_tables-events) +- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics that are listed in the `ProfileEvents.Names` column. - `Settings.Names` (Array(String)) — Names of settings that were changed when the client ran the query. To enable logging changes to settings, set the `log_query_settings` parameter to 1. - `Settings.Values` (Array(String)) — Values of settings that are listed in the `Settings.Names` column. @@ -485,6 +481,71 @@ When the table is deleted manually, it will be automatically created on the fly. You can specify an arbitrary partitioning key for the `system.query_log` table in the [query_log](server_settings/settings.md#server_settings-query-log) server setting (see the `partition_by` parameter). +## system.query_thread_log {#system_tables-query-thread-log} + +The table contains information about each query execution thread. + +ClickHouse creates this table only if the [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) server parameter is specified. This parameter sets the logging rules, such as the logging interval or the name of the table the queries will be logged in. + +To enable query logging, set the [log_query_threads](settings/settings.md#settings-log-query-threads) parameter to 1. For details, see the [Settings](settings/settings.md) section. + +Columns: + +- `event_date` (Date) — the date when the thread has finished execution of the query. +- `event_time` (DateTime) — the date and time when the thread has finished execution of the query. +- `query_start_time` (DateTime) — Start time of query execution. +- `query_duration_ms` (UInt64) — Duration of query execution. +- `read_rows` (UInt64) — Number of read rows. +- `read_bytes` (UInt64) — Number of read bytes. +- `written_rows` (UInt64) — For `INSERT` queries, the number of written rows. For other queries, the column value is 0. +- `written_bytes` (UInt64) — For `INSERT` queries, the number of written bytes. For other queries, the column value is 0. +- `memory_usage` (Int64) — The difference between the amount of allocated and freed memory in context of this thread. +- `peak_memory_usage` (Int64) — The maximum difference between the amount of allocated and freed memory in context of this thread. +- `thread_name` (String) — Name of the thread. +- `thread_number` (UInt32) — Internal thread ID. +- `os_thread_id` (Int32) — OS thread ID. +- `master_thread_number` (UInt32) — Internal ID of initial thread. +- `master_os_thread_id` (Int32) — OS initial ID of initial thread. +- `query` (String) — Query string. +- `is_initial_query` (UInt8) — Query type. Possible values: + - 1 — Query was initiated by the client. + - 0 — Query was initiated by another query for distributed query execution. +- `user` (String) — Name of the user who initiated the current query. +- `query_id` (String) — ID of the query. +- `address` (IPv6) — IP address that was used to make the query. +- `port` (UInt16) — The client port that was used to make the query. +- `initial_user` (String) — Name of the user who ran the initial query (for distributed query execution). +- `initial_query_id` (String) — ID of the initial query (for distributed query execution). +- `initial_address` (IPv6) — IP address that the parent query was launched from. +- `initial_port` (UInt16) — The client port that was used to make the parent query. +- `interface` (UInt8) — Interface that the query was initiated from. Possible values: + - 1 — TCP. + - 2 — HTTP. +- `os_user` (String) — OS's username who runs [clickhouse-client](../interfaces/cli.md). +- `client_hostname` (String) — Hostname of the client machine where the [clickhouse-client](../interfaces/cli.md) or another TCP client is run. +- `client_name` (String) — The [clickhouse-client](../interfaces/cli.md) or another TCP client name. +- `client_revision` (UInt32) — Revision of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_major` (UInt32) — Major version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_minor` (UInt32) — Minor version of the [clickhouse-client](../interfaces/cli.md) or another TCP client. +- `client_version_patch` (UInt32) — Patch component of the [clickhouse-client](../interfaces/cli.md) or another TCP client version. +- `http_method` (UInt8) — HTTP method that initiated the query. Possible values: + - 0 — The query was launched from the TCP interface. + - 1 — `GET` method was used. + - 2 — `POST` method was used. +- `http_user_agent` (String) — The `UserAgent` header passed in the HTTP request. +- `quota_key` (String) — The "quota key" specified in the [quotas](quotas.md) setting (see `keyed`). +- `revision` (UInt32) — ClickHouse revision. +- `ProfileEvents.Names` (Array(String)) — Counters that measure different metrics for this thread. The description of them could be found in the table [system.events](#system_tables-events) +- `ProfileEvents.Values` (Array(UInt64)) — Values of metrics for this thread that are listed in the `ProfileEvents.Names` column. + +By default, logs are added to the table at intervals of 7.5 seconds. You can set this interval in the [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) server setting (see the `flush_interval_milliseconds` parameter). To flush the logs forcibly from the memory buffer into the table, use the `SYSTEM FLUSH LOGS` query. + +When the table is deleted manually, it will be automatically created on the fly. Note that all the previous logs will be deleted. + +!!! note + The storage period for logs is unlimited. Logs aren't automatically deleted from the table. You need to organize the removal of outdated logs yourself. + +You can specify an arbitrary partitioning key for the `system.query_thread_log` table in the [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) server setting (see the `partition_by` parameter). ## system.trace_log {#system_tables-trace_log} diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 2454df4042e..ab194deff15 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -682,7 +682,7 @@ SELECT arrayDifference([0, 10000000000000000000]) ## arrayDistinct(arr) {#array_functions-arraydistinct} -Takes an array, returns an array containing the distinct elements. +Takes an array, returns an array containing the distinct elements. Example: @@ -698,7 +698,7 @@ SELECT arrayDistinct([1, 2, 2, 3, 1]) ## arrayEnumerateDense(arr) {#array_functions-arrayenumeratedense} -Returns an array of the same size as the source array, indicating where each element first appears in the source array. +Returns an array of the same size as the source array, indicating where each element first appears in the source array. Example: @@ -772,22 +772,6 @@ SELECT arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10]) └─────────────────────────────────────────────────────────────┘ ``` -## arrayFlatten(arr) {#array_functions-arrayflatten} - -The `arrayFlatten` (or `flatten` alias) method will collapse the elements of an array to create a single array. - -Example: - -```sql -SELECT arrayFlatten([[1, 2, 3], [4, 5]]) -``` - -```text -┌─arrayFlatten([[1, 2, 3], [4, 5]])─┐ -│ [1,2,3,4,5] │ -└───────────────────────────────────┘ -``` - ## arrayReverse(arr) {#array_functions-arrayreverse} Returns an array of the same size as the original array containing the elements in reverse order. @@ -808,22 +792,78 @@ SELECT arrayReverse([1, 2, 3]) Synonym for ["arrayReverse"](#array_functions-arrayreverse) -[Original article](https://clickhouse.yandex/docs/en/query_language/functions/array_functions/) +## arrayFlatten {#arrayflatten} -## arrayCompact(arr) {#array_functions-arraycompact} +Converts array of arrays to a flat array. -Takes an array, returns an array with consecutive duplicate elements removed. +Function: -Example: +- Applies for any depth of nested arrays, but all the elements should lay at the same level. + + For example, the `[[[1]], [[2], [3]]]` array can be flattened, but the `[[1], [[2], [3]]]` array can't be flattened. + +- Does not change arrays that are already flat. + +The flattened array contains all the elements from all source arrays. + +**Syntax** ```sql -SELECT arrayCompact([1, 2, 2, 3, 2, 3, 3]) +flatten(array_of_arrays) ``` +Alias: `flatten`. + + +**Parameters** + +- `array_of_arrays` — [Array](../../data_types/array.md) of arrays. For example, `[[1,2,3], [4,5]]`. + +**Examples** + +```sql +SELECT flatten([[[1]], [[2], [3]]]) +``` +```text +┌─flatten(array(array([1]), array([2], [3])))─┐ +│ [1,2,3] │ +└─────────────────────────────────────────────┘ +``` + +## arrayCompact {#arraycompact} + +Removes consecutive duplicate elements from an array. The order of result values is determined by the order in the source array. + +**Syntax** + +```sql +arrayCompact(arr) +``` + +**Parameters** + +`arr` — The [array](../../data_types/array.md) to inspect. + +**Returned value** + +The array without duplicate. + +Type: `Array`. + +**Example** + +Query: + +```sql +SELECT arrayCompact([1, 1, nan, nan, 2, 3, 3, 3]) +``` + +Result: + ```text -┌─arrayCompact([1, 2, 2, 3, 2, 3, 3])──┐ -│ [1,2,3,2,3] │ -└──────────────────────────────────────┘ +┌─arrayCompact([1, 1, nan, nan, 2, 3, 3, 3])─┐ +│ [1,nan,nan,2,3] │ +└────────────────────────────────────────────┘ ``` -## \ No newline at end of file +[Original article](https://clickhouse.yandex/docs/en/query_language/functions/array_functions/) diff --git a/docs/en/query_language/functions/hash_functions.md b/docs/en/query_language/functions/hash_functions.md index 2d21d2290ad..d98c56cd584 100644 --- a/docs/en/query_language/functions/hash_functions.md +++ b/docs/en/query_language/functions/hash_functions.md @@ -179,6 +179,8 @@ SELECT farmHash64(array('e','x','a'), 'mple', 10, toDateTime('2019-06-15 23:00:0 Calculates [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452) from a string. This hash function is neither fast nor having a good quality. The only reason to use it is when this algorithm is already used in another system and you have to calculate exactly the same result. +**Syntax** + ```sql SELECT javaHash(''); ``` @@ -187,8 +189,6 @@ SELECT javaHash(''); A `Int32` data type hash value. -Type: `javaHash`. - **Example** Query: @@ -205,15 +205,40 @@ Result: └───────────────────────────┘ ``` -## javaHashUTF16LE +## javaHashUTF16LE {#javahashutf16le} -The same as [JavaHash](#hash_functions-javahash), but for UTF-16LE code points. Works under the assumption that the string contains a set of bytes representing a UTF-16LE encoded text. If this assumption is not met, it returns some result (It only throws an exception in partial cases). +Calculates [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452) from a string, assuming it contains bytes representing a string in UTF-16LE encoding. +**Syntax** + +```sql +javaHashUTF16LE(stringUtf16le) +``` + +**Parameters** + +- `stringUtf16le` — a string in UTF-16LE encoding. + +**Returned value** + +A `Int32` data type hash value. **Example** +Correct query with UTF-16LE encoded string. + +Query: + ```sql -SELECT javaHashUTF16LE(convertCharset('Hello, world!', 'utf-8', 'utf-16le')) +SELECT javaHashUTF16LE(convertCharset('test', 'utf-8', 'utf-16le')) +``` + +Result: + +```text +┌─javaHashUTF16LE(convertCharset('test', 'utf-8', 'utf-16le'))─┐ +│ 3556498 │ +└──────────────────────────────────────────────────────────────┘ ``` ## hiveHash {#hash_functions-hivehash} diff --git a/docs/en/query_language/operators.md b/docs/en/query_language/operators.md index b9780bb7d1d..a12d097b8e5 100644 --- a/docs/en/query_language/operators.md +++ b/docs/en/query_language/operators.md @@ -7,7 +7,7 @@ Groups of operators are listed in order of priority (the higher it is in the lis `a[N]` Access to an element of an array; ` arrayElement(a, N) function`. -`a.N` – Access to a tuble element; `tupleElement(a, N)` function. +`a.N` – Access to a tuple element; `tupleElement(a, N)` function. ## Numeric Negation Operator diff --git a/docs/en/query_language/system.md b/docs/en/query_language/system.md index 229532d0f5e..0b08183afe8 100644 --- a/docs/en/query_language/system.md +++ b/docs/en/query_language/system.md @@ -3,7 +3,7 @@ - [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries) - [RELOAD DICTIONARY](#query_language-system-reload-dictionary) - [DROP DNS CACHE](#query_language-system-drop-dns-cache) -- [DROP MARKS CACHE](#query_language-system-drop-marks-cache) +- [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [FLUSH LOGS](#query_language-system-flush_logs) - [RELOAD CONFIG](#query_language-system-reload-config) - [SHUTDOWN](#query_language-system-shutdown) @@ -36,7 +36,7 @@ Resets ClickHouse's internal DNS cache. Sometimes (for old ClickHouse versions) For more convenient (automatic) cache management, see disable_internal_dns_cache, dns_cache_update_period parameters. -## DROP MARKS CACHE {#query_language-system-drop-marks-cache} +## DROP MARK CACHE {#query_language-system-drop-mark-cache} Resets the mark cache. Used in development of ClickHouse and performance tests. diff --git a/docs/ru/development/build_cross.md b/docs/ru/development/build_cross.md deleted file mode 120000 index f595f252de3..00000000000 --- a/docs/ru/development/build_cross.md +++ /dev/null @@ -1 +0,0 @@ -../../en/development/build_cross.md \ No newline at end of file diff --git a/docs/ru/development/build_cross_osx.md b/docs/ru/development/build_cross_osx.md new file mode 120000 index 00000000000..72e64e8631f --- /dev/null +++ b/docs/ru/development/build_cross_osx.md @@ -0,0 +1 @@ +../../en/development/build_cross_osx.md \ No newline at end of file diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 2028828dd7e..801a89af49e 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -226,18 +226,24 @@ ClickHouse использует небольшое подмножество фу ### 4.3. Ограничение числа одновременных скачиваний с реплик. +Дмитрий Григорьев, ВШЭ. Изначально делал Олег Алексеенков, но пока решение не готово, хотя там не так уж много доделывать. ### 4.4. Ограничение сетевой полосы при репликации. +Дмитрий Григорьев, ВШЭ. + ### 4.5. Возможность продолжить передачу куска данных при репликации после сбоя. +Дмитрий Григорьев, ВШЭ. + ### 4.6. p2p передача для GLOBAL подзапросов. ### 4.7. Ленивая загрузка множеств для IN и JOIN с помощью k/v запросов. ### 4.8. Разделить background pool для fetch и merge. +Дмитрий Григорьев, ВШЭ. В очереди. Исправить проблему, что восстанавливающаяся реплика перестаёт мержить. Частично компенсируется 4.3. @@ -458,6 +464,20 @@ Fuzzing тестирование - это тестирование случай 1. Добавление в SQL диалект ClickHouse функций для генерации случайных данных (пример - случайные бинарные строки заданной длины, случайные валидные UTF-8 строки) и "порчи" данных (например, поменять значения случайных бит с заданной частотой). Это будет использовано для тестирования SQL-функций ClickHouse. +Можно добавить функции: +`randomString(length)` +`randomFixedString(length)` + - строка заданной длины с равномерно распределёнными случайными байтами; +`randomStringASCII(length)` +`randomStringUTF8(length)` + +`fuzzBits(s, inverse_probability)` - изменить каждый бит строки на противоположный с заданной вероятностью; +`fuzzBytes(s, inverse_probability)` - изменить каждый байт строки на равномерно случайный с заданной вероятностью; + +У каждой функции опциональный аргумент против склейки одинаковых выражений в запросе. + +Также можно сделать функции с детерминированным генератором случайных чисел (аргументом передаётся seed) для воспроизводимости тестовых кейсов. + ### 7.24. Fuzzing лексера и парсера запросов; кодеков и форматов. Андрей Некрашевич, ВШЭ. @@ -1239,7 +1259,7 @@ zhang2014. Василий Немков, Altinity - временно приостановлено, но намерения остаются в силе. -Мы считаем важным, что код в ClickHouse содержит разумные оптимизации, основанные на анализе производительности. Но иногда бывают досадные исключения. +Мы считаем важным, что код в ClickHouse содержит разумные оптимизации, основанные на анализе производительности. Но иногда бывают досадные исключения. ### 22.17. Консистентно работающий POPULATE для MaterializedView. @@ -1263,6 +1283,9 @@ zhang2014. После 10.14. +https://github.com/ClickHouse/ClickHouse/issues/7237 +https://github.com/ClickHouse/ClickHouse/issues/2655 + ### 22.23. Правильная обработка Nullable в функциях, которые кидают исключение на default значении: modulo, intDiv. ### 22.24. Излишняя фильтрация ODBC connection string. diff --git a/docs/ru/interfaces/cli.md b/docs/ru/interfaces/cli.md index f2040c4af1b..a67ae87f6ab 100644 --- a/docs/ru/interfaces/cli.md +++ b/docs/ru/interfaces/cli.md @@ -1,17 +1,23 @@ # Клиент командной строки -Для работы из командной строки вы можете использовать `clickhouse-client`: +ClickHouse предоставляет собственный клиент командной строки: `clickhouse-client`. Клиент поддерживает запуск с аргументами командной строки и с конфигурационными файлами. Подробнее читайте в разделе [Конфигурирование](#interfaces_cli_configuration). + +Клиент [устанавливается](../getting_started/index.md) пакетом `clickhouse-client` и запускается командой `clickhouse-client`. ```bash $ clickhouse-client -ClickHouse client version 0.0.26176. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.26176. +ClickHouse client version 19.17.1.1579 (official build). +Connecting to localhost:9000 as user default. +Connected to ClickHouse server version 19.17.1 revision 54428. :) ``` -Клиент поддерживает параметры командной строки и конфигурационные файлы. Подробнее читайте в разделе "[Конфигурирование](#interfaces_cli_configuration)". +Клиенты и серверы различных версий совместимы, однако если клиент старее сервера, то некоторые новые фукнции могут быть недоступны. Мы рекомендуем использовать одинаковые версии клиента и сервера. При подключении клиента к более новому серверу `clickhouse-client` выводит сообщение: + +``` +ClickHouse client version is older than ClickHouse server. It may lack support for new features. +``` ## Использование {#cli_usage} diff --git a/docs/ru/interfaces/http.md b/docs/ru/interfaces/http.md index c7c32a46a4c..4da101796f1 100644 --- a/docs/ru/interfaces/http.md +++ b/docs/ru/interfaces/http.md @@ -28,8 +28,12 @@ $ wget -O- -q 'http://localhost:8123/?query=SELECT 1' $ echo -ne 'GET /?query=SELECT%201 HTTP/1.0\r\n\r\n' | nc localhost 8123 HTTP/1.0 200 OK +Date: Wed, 27 Nov 2019 10:30:18 GMT Connection: Close -Date: Fri, 16 Nov 2012 19:21:50 GMT +Content-Type: text/tab-separated-values; charset=UTF-8 +X-ClickHouse-Server-Display-Name: clickhouse.ru-central1.internal +X-ClickHouse-Query-Id: 5abe861c-239c-467f-b955-8a201abb8b7f +X-ClickHouse-Summary: {"read_rows":"0","read_bytes":"0","written_rows":"0","written_bytes":"0","total_rows_to_read":"0"} 1 ``` diff --git a/docs/ru/operations/monitoring.md b/docs/ru/operations/monitoring.md index da24c7e960b..248d478506b 100644 --- a/docs/ru/operations/monitoring.md +++ b/docs/ru/operations/monitoring.md @@ -34,4 +34,4 @@ ClickHouse собирает: Также, можно отслеживать доступность сервера через HTTP API. Отправьте `HTTP GET` к ресурсу `/`. Если сервер доступен, он отвечает `200 OK`. -Для мониторинга серверов в кластерной конфигурации необходимо установить параметр [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) и использовать HTTP ресурс `/replicas-delay`. Если реплика доступна и не отстаёт от других реплик, то запрос к `/replicas-delay` возвращает `200 OK`. Если реплика отстаёт, то она возвращает информацию о размере отставания. +Для мониторинга серверов в кластерной конфигурации необходимо установить параметр [max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries) и использовать HTTP ресурс `/replicas_status`. Если реплика доступна и не отстаёт от других реплик, то запрос к `/replicas_status` возвращает `200 OK`. Если реплика отстаёт, то она возвращает информацию о размере отставания. diff --git a/docs/ru/operations/server_settings/settings.md b/docs/ru/operations/server_settings/settings.md index aca2fed8063..ca1c255bee3 100644 --- a/docs/ru/operations/server_settings/settings.md +++ b/docs/ru/operations/server_settings/settings.md @@ -580,6 +580,33 @@ ClickHouse проверит условия `min_part_size` и `min_part_size_rat ``` +## query_thread_log {#server_settings-query-thread-log} + +Настройка логирования потоков выполнения запросов, принятых с настройкой [log_query_threads=1](../settings/settings.md#settings-log-query-threads). + +Запросы логируются не в отдельный файл, а в системную таблицу [system.query_thread_log](../system_tables.md#system_tables-query-thread-log). Вы можете изменить название этой таблицы в параметре `table` (см. ниже). + +При настройке логирования используются следующие параметры: + +- `database` — имя базы данных; +- `table` — имя таблицы, куда будет записываться лог; +- `partition_by` — [произвольный ключ партиционирования](../../operations/table_engines/custom_partitioning_key.md) для таблицы с логами; +- `flush_interval_milliseconds` — период сброса данных из буфера в памяти в таблицу. + +Если таблица не существует, то ClickHouse создаст её. Если структура журнала запросов изменилась при обновлении сервера ClickHouse, то таблица со старой структурой переименовывается, а новая таблица создается автоматически. + +**Пример** + +```xml + + system + query_thread_log
+ toMonday(event_date) + 7500 +
+``` + + ## remote_servers {#server_settings_remote_servers} Конфигурация кластеров, которые использует движок таблиц [Distributed](../../operations/table_engines/distributed.md) и табличная функция `cluster`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 81cff172f98..e0045bd42ef 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -488,6 +488,16 @@ ClickHouse использует этот параметр при чтении д log_queries=1 +## log_query_threads {#settings-log-query-threads} + +Установка логирования информации о потоках выполнения запроса. + +Лог информации о потоках выполнения запросов, переданных в ClickHouse с этой установкой, записывается согласно правилам конфигурационного параметра сервера [query_thread_log](../server_settings/settings.md#server_settings-query-thread-log). + +**Пример** : + + log_query_threads=1 + ## max_insert_block_size {#settings-max_insert_block_size} Формировать блоки указанного размера, при вставке в таблицу. @@ -570,6 +580,12 @@ Default value: 10000 seconds. Значение по умолчанию: 10, 300, 300. +## cancel_http_readonly_queries_on_client_close + +Отменяет HTTP readonly запросы (например, SELECT), когда клиент обрывает соединение до завершения получения данных. + +Значение по умолчанию: 0 + ## poll_interval Блокироваться в цикле ожидания запроса в сервере на указанное количество секунд. diff --git a/docs/ru/operations/system_tables.md b/docs/ru/operations/system_tables.md index fa6c80bfb05..0e4ffc8e056 100644 --- a/docs/ru/operations/system_tables.md +++ b/docs/ru/operations/system_tables.md @@ -410,8 +410,8 @@ ClickHouse создаёт таблицу только в том случае, к - `'QueryFinish' = 2` — успешное завершение выполнения запроса. - `'ExceptionBeforeStart' = 3` — исключение перед началом обработки запроса. - `'ExceptionWhileProcessing' = 4` — исключение во время обработки запроса. -- `event_date` (Date) — дата события. -- `event_time` (DateTime) — время события. +- `event_date` (Date) — дата начала запроса. +- `event_time` (DateTime) — время начала запроса. - `query_start_time` (DateTime) — время начала обработки запроса. - `query_duration_ms` (UInt64) — длительность обработки запроса. - `read_rows` (UInt64) — количество прочитанных строк. @@ -421,43 +421,39 @@ ClickHouse создаёт таблицу только в том случае, к - `result_rows` (UInt64) — количество строк в результате. - `result_bytes` (UInt64) — объём результата в байтах. - `memory_usage` (UInt64) — потребление RAM запросом. -- `query` (String) — строка запроса. -- `exception` (String) — сообщение исключения. +- `query` (String) — текст запроса. +- `exception` (String) — сообщение исключения, если запрос завершился по исключению. - `stack_trace` (String) — трассировка (список функций, последовательно вызванных перед ошибкой). Пустая строка, если запрос успешно завершен. - `is_initial_query` (UInt8) — вид запроса. Возможные значения: - 1 — запрос был инициирован клиентом. - 0 — запрос был инициирован другим запросом при распределенном запросе. - `user` (String) — пользователь, запустивший текущий запрос. - `query_id` (String) — ID запроса. -- `address` (FixedString(16)) — IP адрес, с которого пришел запрос. -- `port` (UInt16) — порт, на котором сервер принял запрос. +- `address` (IPv6) — IP адрес, с которого пришел запрос. +- `port` (UInt16) — порт, с которого клиент сделал запрос - `initial_user` (String) — пользователь, запустивший первоначальный запрос (для распределенных запросов). - `initial_query_id` (String) — ID родительского запроса. -- `initial_address` (FixedString(16)) — IP адрес, с которого пришел родительский запрос. -- `initial_port` (UInt16) — порт, на котором сервер принял родительский запрос от клиента. +- `initial_address` (IPv6) — IP адрес, с которого пришел родительский запрос. +- `initial_port` (UInt16) — порт, с которого клиент сделал родительский запрос. - `interface` (UInt8) — интерфейс, с которого ушёл запрос. Возможные значения: - 1 — TCP. - 2 — HTTP. -- `os_user` (String) — операционная система пользователя. -- `client_hostname` (String) — имя сервера, к которому присоединился [clickhouse-client](../interfaces/cli.md). -- `client_name` (String) — [clickhouse-client](../interfaces/cli.md). -- `client_revision` (UInt32) — ревизия [clickhouse-client](../interfaces/cli.md). -- `client_version_major` (UInt32) — старшая версия [clickhouse-client](../interfaces/cli.md). -- `client_version_minor` (UInt32) — младшая версия [clickhouse-client](../interfaces/cli.md). -- `client_version_patch` (UInt32) — патч [clickhouse-client](../interfaces/cli.md). +- `os_user` (String) — имя пользователя в OS, который запустил [clickhouse-client](../interfaces/cli.md). +- `client_hostname` (String) — имя сервера, с которого присоединился [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. +- `client_name` (String) — [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. +- `client_revision` (UInt32) — ревизия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_major` (UInt32) — старшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_minor` (UInt32) — младшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_patch` (UInt32) — патч [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. - `http_method` (UInt8) — HTTP метод, инициировавший запрос. Возможные значения: - 0 — запрос запущен с интерфейса TCP. - 1 — `GET`. - 2 — `POST`. - `http_user_agent` (String) — HTTP заголовок `UserAgent`. -- `quota_key` (String) — идентификатор квоты из настроек [квот](quotas.md). +- `quota_key` (String) — "ключ квоты" из настроек [квот](quotas.md) (см. `keyed`). - `revision` (UInt32) — ревизия ClickHouse. - `thread_numbers` (Array(UInt32)) — количество потоков, участвующих в обработке запросов. -- `ProfileEvents.Names` (Array(String)) — Счетчики для изменения метрик: - - Время, потраченное на чтение и запись по сети. - - Время, потраченное на чтение и запись на диск. - - Количество сетевых ошибок. - - Время, потраченное на ожидание, когда пропускная способность сети ограничена. +- `ProfileEvents.Names` (Array(String)) — Счетчики для изменения различных метрик. Описание метрик можно получить из таблицы [system.events](#system_tables-events - `ProfileEvents.Values` (Array(UInt64)) — метрики, перечисленные в столбце `ProfileEvents.Names`. - `Settings.Names` (Array(String)) — имена настроек, которые меняются, когда клиент выполняет запрос. Чтобы разрешить логирование изменений настроек, установите параметр `log_query_settings` равным 1. - `Settings.Values` (Array(String)) — Значения настроек, которые перечислены в столбце `Settings.Names`. @@ -477,6 +473,72 @@ ClickHouse создаёт таблицу только в том случае, к Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query_log](server_settings/settings.md#server_settings-query-log) (параметр `partition_by`). +## system.query_thread_log {#system_tables-query-thread-log} + +Содержит информацию о каждом потоке выполняемых запросов. + +ClickHouse создаёт таблицу только в том случае, когда установлен конфигурационный параметр сервера [query_thread_log](server_settings/settings.md#server_settings-query-thread-log). Параметр задаёт правила ведения лога, такие как интервал логирования или имя таблицы, в которую будут логгироваться запросы. + +Чтобы включить логирование, задайте значение параметра [log_query_threads](settings/settings.md#settings-log-query-threads) равным 1. Подробности смотрите в разделе [Настройки](settings/settings.md). + +Столбцы: + +- `event_date` (Date) — дата завершения выполнения запроса потоком. +- `event_time` (DateTime) — дата и время завершения выполнения запроса потоком. +- `query_start_time` (DateTime) — время начала обработки запроса. +- `query_duration_ms` (UInt64) — длительность обработки запроса в миллисекундах. +- `read_rows` (UInt64) — количество прочитанных строк. +- `read_bytes` (UInt64) — количество прочитанных байтов. +- `written_rows` (UInt64) — количество записанных строк для запросов `INSERT`. Для других запросов, значение столбца 0. +- `written_bytes` (UInt64) — объем записанных данных в байтах для запросов `INSERT`. Для других запросов, значение столбца 0. +- `memory_usage` (Int64) — разница между выделенной и освобождённой памятью в контексте потока. +- `peak_memory_usage` (Int64) — максимальная разница между выделенной и освобождённой памятью в контексте потока. +- `thread_name` (String) — Имя потока. +- `thread_number` (UInt32) — Внутренний ID потока. +- `os_thread_id` (Int32) — tid (ID потока операционной системы). +- `master_thread_number` (UInt32) — Внутренний ID главного потока. +- `master_os_thread_id` (Int32) — tid (ID потока операционной системы) главного потока. +- `query` (String) — текст запроса. +- `is_initial_query` (UInt8) — вид запроса. Возможные значения: + - 1 — запрос был инициирован клиентом. + - 0 — запрос был инициирован другим запросом при распределенном запросе. +- `user` (String) — пользователь, запустивший текущий запрос. +- `query_id` (String) — ID запроса. +- `address` (IPv6) — IP адрес, с которого пришел запрос. +- `port` (UInt16) — порт, с которого пришел запрос. +- `initial_user` (String) — пользователь, запустивший первоначальный запрос (для распределенных запросов). +- `initial_query_id` (String) — ID родительского запроса. +- `initial_address` (IPv6) — IP адрес, с которого пришел родительский запрос. +- `initial_port` (UInt16) — порт, пришел родительский запрос. +- `interface` (UInt8) — интерфейс, с которого ушёл запрос. Возможные значения: + - 1 — TCP. + - 2 — HTTP. +- `os_user` (String) — имя пользователя в OS, который запустил [clickhouse-client](../interfaces/cli.md). +- `client_hostname` (String) — hostname клиентской машины, с которой присоединился [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. +- `client_name` (String) — [clickhouse-client](../interfaces/cli.md) или другой TCP клиент. +- `client_revision` (UInt32) — ревизия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_major` (UInt32) — старшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_minor` (UInt32) — младшая версия [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `client_version_patch` (UInt32) — патч [clickhouse-client](../interfaces/cli.md) или другого TCP клиента. +- `http_method` (UInt8) — HTTP метод, инициировавший запрос. Возможные значения: + - 0 — запрос запущен с интерфейса TCP. + - 1 — `GET`. + - 2 — `POST`. +- `http_user_agent` (String) — HTTP заголовок `UserAgent`. +- `quota_key` (String) — "ключ квоты" из настроек [квот](quotas.md) (см. `keyed`). +- `revision` (UInt32) — ревизия ClickHouse. +- `ProfileEvents.Names` (Array(String)) — Счетчики для изменения различных метрик для данного потока. Описание метрик можно получить из таблицы [system.events](#system_tables-events +- `ProfileEvents.Values` (Array(UInt64)) — метрики для данного потока, перечисленные в столбце `ProfileEvents.Names`. + +По умолчанию, строки добавляются в таблицу логирования с интервалом в 7,5 секунд. Можно задать интервал в конфигурационном параметре сервера [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) (смотрите параметр `flush_interval_milliseconds`). Чтобы принудительно записать логи из буффера памяти в таблицу, используйте запрос `SYSTEM FLUSH LOGS`. + +Если таблицу удалить вручную, она пересоздастся автоматически "на лету". При этом все логи на момент удаления таблицы будут удалены. + +!!! note "Примечание" + Срок хранения логов не ограничен. Логи не удаляются из таблицы автоматически. Вам необходимо самостоятельно организовать удаление устаревших логов. + +Можно указать произвольный ключ партиционирования для таблицы `system.query_log` в конфигурации [query_thread_log](server_settings/settings.md#server_settings-query-thread-log) (параметр `partition_by`). + ## system.replicas {#system_tables-replicas} Содержит информацию и статус для реплицируемых таблиц, расположенных на локальном сервере. diff --git a/docs/ru/query_language/functions/array_functions.md b/docs/ru/query_language/functions/array_functions.md index 377750c99c1..f12395ca64d 100644 --- a/docs/ru/query_language/functions/array_functions.md +++ b/docs/ru/query_language/functions/array_functions.md @@ -814,4 +814,40 @@ SELECT arrayReverse([1, 2, 3]) Синоним для ["arrayReverse"](#array_functions-arrayreverse) +## arrayCompact {#arraycompact} + +Удаляет дубликаты из массива. Порядок результирующих значений определяется порядком в исходном массиве. + +**Синтаксис** + +```sql +arrayCompact(arr) +``` + +**Параметры** + +`arr` — [Массив](../../data_types/array.md) для обхода. + +**Возвращаемое значение** + +Массив без дубликатов. + +Тип: `Array`. + +**Пример** + +Запрос: + +```sql +SELECT arrayCompact([1, 1, nan, nan, 2, 3, 3, 3]) +``` + +Ответ: + +```text +┌─arrayCompact([1, 1, nan, nan, 2, 3, 3, 3])─┐ +│ [1,nan,nan,2,3] │ +└────────────────────────────────────────────┘ +``` + [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/array_functions/) diff --git a/docs/ru/query_language/functions/hash_functions.md b/docs/ru/query_language/functions/hash_functions.md index f7d2237a071..47384e78565 100644 --- a/docs/ru/query_language/functions/hash_functions.md +++ b/docs/ru/query_language/functions/hash_functions.md @@ -207,6 +207,44 @@ SELECT javaHash('Hello, world!'); └───────────────────────────┘ ``` +## javaHashUTF16LE {#javahashutf16le} + +Вычисляет [JavaHash](http://hg.openjdk.java.net/jdk8u/jdk8u/jdk/file/478a4add975b/src/share/classes/java/lang/String.java#l1452) от строки, при допущении, что строка представлена в кодировке `UTF-16LE`. + +**Синтаксис** + +```sql +javaHashUTF16LE(stringUtf16le) +``` + +**Параметры** + +- `stringUtf16le` — строка в `UTF-16LE`. + +**Возвращаемое значение** + +Хэш-значение типа `Int32`. + +Тип: `javaHash`. + +**Пример** + +Верный запрос для строки кодированной в `UTF-16LE`. + +Запрос: + +```sql +SELECT javaHashUTF16LE(convertCharset('test', 'utf-8', 'utf-16le')) +``` + +Ответ: + +```text +┌─javaHashUTF16LE(convertCharset('test', 'utf-8', 'utf-16le'))─┐ +│ 3556498 │ +└──────────────────────────────────────────────────────────────┘ +``` + ## hiveHash {#hash_functions-hivehash} Вычисляет `HiveHash` от строки. diff --git a/docs/ru/query_language/system.md b/docs/ru/query_language/system.md index 3457a87e188..31e0c3cf90b 100644 --- a/docs/ru/query_language/system.md +++ b/docs/ru/query_language/system.md @@ -3,7 +3,7 @@ - [RELOAD DICTIONARIES](#query_language-system-reload-dictionaries) - [RELOAD DICTIONARY](#query_language-system-reload-dictionary) - [DROP DNS CACHE](#query_language-system-drop-dns-cache) -- [DROP MARKS CACHE](#query_language-system-drop-marks-cache) +- [DROP MARK CACHE](#query_language-system-drop-mark-cache) - [FLUSH LOGS](#query_language-system-flush_logs) - [RELOAD CONFIG](#query_language-system-reload-config) - [SHUTDOWN](#query_language-system-shutdown) @@ -36,7 +36,7 @@ SELECT name, status FROM system.dictionaries; Для более удобного (автоматического) управления кешем см. параметры disable_internal_dns_cache, dns_cache_update_period. -## DROP MARKS CACHE {#query_language-system-drop-marks-cache} +## DROP MARK CACHE {#query_language-system-drop-mark-cache} Сбрасывает кеш "засечек" (`mark cache`). Используется при разработке ClickHouse и тестах производительности. diff --git a/docs/toc_en.yml b/docs/toc_en.yml index 3c4a5506a06..d2b50c7f421 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -216,7 +216,7 @@ nav: - 'How to Build ClickHouse on Linux': 'development/build.md' - 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md' - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' - - 'How to Write C++ code': 'development/style.md' + - 'How to Write C++ Code': 'development/style.md' - 'How to Run ClickHouse Tests': 'development/tests.md' - 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md' - 'Third-Party Libraries Used': 'development/contrib.md' diff --git a/docs/toc_ja.yml b/docs/toc_ja.yml new file mode 100644 index 00000000000..3c4a5506a06 --- /dev/null +++ b/docs/toc_ja.yml @@ -0,0 +1,230 @@ +nav: + +- 'Introduction': + - 'Overview': 'index.md' + - 'Distinctive Features of ClickHouse': 'introduction/distinctive_features.md' + - 'ClickHouse Features that Can Be Considered Disadvantages': 'introduction/features_considered_disadvantages.md' + - 'Performance': 'introduction/performance.md' + - 'The Yandex.Metrica Task': 'introduction/ya_metrika_task.md' + +- 'Getting Started': + - 'Deploying and Running': 'getting_started/index.md' + - 'Example Datasets': + - 'OnTime': 'getting_started/example_datasets/ontime.md' + - 'New York Taxi Data': 'getting_started/example_datasets/nyc_taxi.md' + - 'AMPLab Big Data Benchmark': 'getting_started/example_datasets/amplab_benchmark.md' + - 'WikiStat': 'getting_started/example_datasets/wikistat.md' + - 'Terabyte Click Logs from Criteo': 'getting_started/example_datasets/criteo.md' + - 'Star Schema Benchmark': 'getting_started/example_datasets/star_schema.md' + - 'Yandex.Metrica Data': 'getting_started/example_datasets/metrica.md' + +- 'Interfaces': + - 'Introduction': 'interfaces/index.md' + - 'Command-Line Client': 'interfaces/cli.md' + - 'Native Interface (TCP)': 'interfaces/tcp.md' + - 'HTTP Interface': 'interfaces/http.md' + - 'Input and Output Formats': 'interfaces/formats.md' + - 'JDBC Driver': 'interfaces/jdbc.md' + - 'ODBC Driver': 'interfaces/odbc.md' + - 'C++ Client Library': 'interfaces/cpp.md' + - 'Third-Party': + - 'Client Libraries': 'interfaces/third-party/client_libraries.md' + - 'Integrations': 'interfaces/third-party/integrations.md' + - 'Visual Interfaces': 'interfaces/third-party/gui.md' + - 'Proxies': 'interfaces/third-party/proxy.md' + +- 'Database Engines': + - 'Introduction': 'database_engines/index.md' + - 'MySQL': 'database_engines/mysql.md' + - 'Lazy': 'database_engines/lazy.md' + +- 'Table Engines': + - 'Introduction': 'operations/table_engines/index.md' + - 'MergeTree Family': + - 'MergeTree': 'operations/table_engines/mergetree.md' + - 'Data Replication': 'operations/table_engines/replication.md' + - 'Custom Partitioning Key': 'operations/table_engines/custom_partitioning_key.md' + - 'ReplacingMergeTree': 'operations/table_engines/replacingmergetree.md' + - 'SummingMergeTree': 'operations/table_engines/summingmergetree.md' + - 'AggregatingMergeTree': 'operations/table_engines/aggregatingmergetree.md' + - 'CollapsingMergeTree': 'operations/table_engines/collapsingmergetree.md' + - 'VersionedCollapsingMergeTree': 'operations/table_engines/versionedcollapsingmergetree.md' + - 'GraphiteMergeTree': 'operations/table_engines/graphitemergetree.md' + - 'Log Family': + - 'Introduction': 'operations/table_engines/log_family.md' + - 'StripeLog': 'operations/table_engines/stripelog.md' + - 'Log': 'operations/table_engines/log.md' + - 'TinyLog': 'operations/table_engines/tinylog.md' + - 'Integrations': + - 'Kafka': 'operations/table_engines/kafka.md' + - 'MySQL': 'operations/table_engines/mysql.md' + - 'JDBC': 'operations/table_engines/jdbc.md' + - 'ODBC': 'operations/table_engines/odbc.md' + - 'HDFS': 'operations/table_engines/hdfs.md' + - 'Special': + - 'Distributed': 'operations/table_engines/distributed.md' + - 'External data': 'operations/table_engines/external_data.md' + - 'Dictionary': 'operations/table_engines/dictionary.md' + - 'Merge': 'operations/table_engines/merge.md' + - 'File': 'operations/table_engines/file.md' + - 'Null': 'operations/table_engines/null.md' + - 'Set': 'operations/table_engines/set.md' + - 'Join': 'operations/table_engines/join.md' + - 'URL': 'operations/table_engines/url.md' + - 'View': 'operations/table_engines/view.md' + - 'MaterializedView': 'operations/table_engines/materializedview.md' + - 'Memory': 'operations/table_engines/memory.md' + - 'Buffer': 'operations/table_engines/buffer.md' + +- 'SQL Reference': + - 'hidden': 'query_language/index.md' + - 'Syntax': 'query_language/syntax.md' + - 'Statements': + - 'SELECT': 'query_language/select.md' + - 'INSERT INTO': 'query_language/insert_into.md' + - 'CREATE': 'query_language/create.md' + - 'ALTER': 'query_language/alter.md' + - 'SYSTEM': 'query_language/system.md' + - 'SHOW': 'query_language/show.md' + - 'Other': 'query_language/misc.md' + - 'Functions': + - 'Introduction': 'query_language/functions/index.md' + - 'Arithmetic': 'query_language/functions/arithmetic_functions.md' + - 'Comparison': 'query_language/functions/comparison_functions.md' + - 'Logical': 'query_language/functions/logical_functions.md' + - 'Type Conversion': 'query_language/functions/type_conversion_functions.md' + - 'Working with Dates and Times': 'query_language/functions/date_time_functions.md' + - 'Working with strings': 'query_language/functions/string_functions.md' + - 'For Searching Strings': 'query_language/functions/string_search_functions.md' + - 'For Replacing in Strings': 'query_language/functions/string_replace_functions.md' + - 'Conditional ': 'query_language/functions/conditional_functions.md' + - 'Mathematical': 'query_language/functions/math_functions.md' + - 'Rounding': 'query_language/functions/rounding_functions.md' + - 'Working with Arrays': 'query_language/functions/array_functions.md' + - 'Splitting and Merging Strings and Arrays': 'query_language/functions/splitting_merging_functions.md' + - 'Bit': 'query_language/functions/bit_functions.md' + - 'Bitmap': 'query_language/functions/bitmap_functions.md' + - 'Hash': 'query_language/functions/hash_functions.md' + - 'Generating Pseudo-Random Numbers': 'query_language/functions/random_functions.md' + - 'Encoding': 'query_language/functions/encoding_functions.md' + - 'Working with UUID': 'query_language/functions/uuid_functions.md' + - 'Working with URLs': 'query_language/functions/url_functions.md' + - 'Working with IP Addresses': 'query_language/functions/ip_address_functions.md' + - 'Working with JSON.': 'query_language/functions/json_functions.md' + - 'Higher-Order': 'query_language/functions/higher_order_functions.md' + - 'Working with External Dictionaries': 'query_language/functions/ext_dict_functions.md' + - 'Working with Yandex.Metrica Dictionaries': 'query_language/functions/ym_dict_functions.md' + - 'Implementing the IN Operator': 'query_language/functions/in_functions.md' + - 'arrayJoin': 'query_language/functions/array_join.md' + - 'Working with geographical coordinates': 'query_language/functions/geo.md' + - 'Working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' + - 'Machine Learning Functions': 'query_language/functions/machine_learning_functions.md' + - 'Other': 'query_language/functions/other_functions.md' + - 'Aggregate Functions': + - 'Introduction': 'query_language/agg_functions/index.md' + - 'Reference': 'query_language/agg_functions/reference.md' + - 'Aggregate function combinators': 'query_language/agg_functions/combinators.md' + - 'Parametric aggregate functions': 'query_language/agg_functions/parametric_functions.md' + - 'Table Functions': + - 'Introduction': 'query_language/table_functions/index.md' + - 'file': 'query_language/table_functions/file.md' + - 'merge': 'query_language/table_functions/merge.md' + - 'numbers': 'query_language/table_functions/numbers.md' + - 'remote': 'query_language/table_functions/remote.md' + - 'url': 'query_language/table_functions/url.md' + - 'mysql': 'query_language/table_functions/mysql.md' + - 'jdbc': 'query_language/table_functions/jdbc.md' + - 'odbc': 'query_language/table_functions/odbc.md' + - 'hdfs': 'query_language/table_functions/hdfs.md' + - 'input': 'query_language/table_functions/input.md' + - 'Dictionaries': + - 'Introduction': 'query_language/dicts/index.md' + - 'External Dictionaries': + - 'General Description': 'query_language/dicts/external_dicts.md' + - 'Configuring an External Dictionary': 'query_language/dicts/external_dicts_dict.md' + - 'Storing Dictionaries in Memory': 'query_language/dicts/external_dicts_dict_layout.md' + - 'Dictionary Updates': 'query_language/dicts/external_dicts_dict_lifetime.md' + - 'Sources of External Dictionaries': 'query_language/dicts/external_dicts_dict_sources.md' + - 'Dictionary Key and Fields': 'query_language/dicts/external_dicts_dict_structure.md' + - 'Internal Dictionaries': 'query_language/dicts/internal_dicts.md' + - 'Operators': 'query_language/operators.md' + - 'Data Types': + - 'Introduction': 'data_types/index.md' + - 'UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64': 'data_types/int_uint.md' + - 'Float32, Float64': 'data_types/float.md' + - 'Decimal': 'data_types/decimal.md' + - 'Boolean': 'data_types/boolean.md' + - 'String': 'data_types/string.md' + - 'FixedString(N)': 'data_types/fixedstring.md' + - 'UUID': 'data_types/uuid.md' + - 'Date': 'data_types/date.md' + - 'DateTime': 'data_types/datetime.md' + - 'Enum': 'data_types/enum.md' + - 'Array(T)': 'data_types/array.md' + - 'AggregateFunction(name, types_of_arguments...)': 'data_types/nested_data_structures/aggregatefunction.md' + - 'Tuple(T1, T2, ...)': 'data_types/tuple.md' + - 'Nullable': 'data_types/nullable.md' + - 'Nested Data Structures': + - 'hidden': 'data_types/nested_data_structures/index.md' + - 'Nested(Name1 Type1, Name2 Type2, ...)': 'data_types/nested_data_structures/nested.md' + - 'Special Data Types': + - 'hidden': 'data_types/special_data_types/index.md' + - 'Expression': 'data_types/special_data_types/expression.md' + - 'Set': 'data_types/special_data_types/set.md' + - 'Nothing': 'data_types/special_data_types/nothing.md' + - 'Interval': 'data_types/special_data_types/interval.md' + - 'Domains': + - 'Overview': 'data_types/domains/overview.md' + - 'IPv4': 'data_types/domains/ipv4.md' + - 'IPv6': 'data_types/domains/ipv6.md' + +- 'Guides': + - 'Overview': 'guides/index.md' + - 'Applying CatBoost Models': 'guides/apply_catboost_model.md' + +- 'Operations': + - 'Introduction': 'operations/index.md' + - 'Requirements': 'operations/requirements.md' + - 'Monitoring': 'operations/monitoring.md' + - 'Troubleshooting': 'operations/troubleshooting.md' + - 'Usage Recommendations': 'operations/tips.md' + - 'ClickHouse Update': 'operations/update.md' + - 'Access Rights': 'operations/access_rights.md' + - 'Data Backup': 'operations/backup.md' + - 'Configuration Files': 'operations/configuration_files.md' + - 'Quotas': 'operations/quotas.md' + - 'System Tables': 'operations/system_tables.md' + - 'Server Configuration Parameters': + - 'Introduction': 'operations/server_settings/index.md' + - 'Server Settings': 'operations/server_settings/settings.md' + - 'Settings': + - 'Introduction': 'operations/settings/index.md' + - 'Permissions for Queries': 'operations/settings/permissions_for_queries.md' + - 'Restrictions on Query Complexity': 'operations/settings/query_complexity.md' + - 'Settings': 'operations/settings/settings.md' + - 'Settings Profiles': 'operations/settings/settings_profiles.md' + - 'Constraints on Settings': 'operations/settings/constraints_on_settings.md' + - 'User Settings': 'operations/settings/settings_users.md' + - 'Utilities': + - 'Overview': 'operations/utils/index.md' + - 'clickhouse-copier': 'operations/utils/clickhouse-copier.md' + - 'clickhouse-local': 'operations/utils/clickhouse-local.md' + +- 'Development': + - 'hidden': 'development/index.md' + - 'Overview of ClickHouse Architecture': 'development/architecture.md' + - 'How to Build ClickHouse on Linux': 'development/build.md' + - 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md' + - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' + - 'How to Write C++ code': 'development/style.md' + - 'How to Run ClickHouse Tests': 'development/tests.md' + - 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md' + - 'Third-Party Libraries Used': 'development/contrib.md' + +- 'What''s New': + - 'Roadmap': 'roadmap.md' + - 'Changelog': 'changelog.md' + - 'Security Changelog': 'security_changelog.md' + +- 'F.A.Q.': + - 'General Questions': 'faq/general.md' diff --git a/docs/tools/build.py b/docs/tools/build.py index 729229fdee7..a76ac845d3d 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -74,6 +74,7 @@ def build_for_lang(lang, args): 'en': 'ClickHouse %s Documentation', 'ru': 'Документация ClickHouse %s', 'zh': 'ClickHouse文档 %s', + 'ja': 'ClickHouseドキュメント %s', 'fa': 'مستندات %sClickHouse' } @@ -241,7 +242,7 @@ if __name__ == '__main__': os.chdir(os.path.join(os.path.dirname(__file__), '..')) arg_parser = argparse.ArgumentParser() - arg_parser.add_argument('--lang', default='en,ru,zh,fa') + arg_parser.add_argument('--lang', default='en,ru,zh,ja,fa') arg_parser.add_argument('--docs-dir', default='.') arg_parser.add_argument('--theme-dir', default='mkdocs-material-theme') arg_parser.add_argument('--website-dir', default=os.path.join('..', 'website')) diff --git a/docs/tools/easy_edit.sh b/docs/tools/easy_edit.sh index 28c38453d0d..ed8a43fead7 100755 --- a/docs/tools/easy_edit.sh +++ b/docs/tools/easy_edit.sh @@ -14,7 +14,7 @@ popd rm -rf "${EDIT_DIR}" || true -for DOCS_LANG in en ru zh fa +for DOCS_LANG in en ru zh ja fa do for ARTICLE in ${ARTICLES} do diff --git a/docs/tools/make_links.sh b/docs/tools/make_links.sh index cca2f5feb6b..084f8b9d97b 100755 --- a/docs/tools/make_links.sh +++ b/docs/tools/make_links.sh @@ -6,7 +6,7 @@ function do_make_links() { - langs=(en ru fa zh) + langs=(en ru zh ja fa) src_file="$1" for lang in "${langs[@]}" do diff --git a/docs/tools/mkdocs-material-theme/assets/flags/ja.svg b/docs/tools/mkdocs-material-theme/assets/flags/ja.svg new file mode 100644 index 00000000000..177d0e78819 --- /dev/null +++ b/docs/tools/mkdocs-material-theme/assets/flags/ja.svg @@ -0,0 +1,5 @@ + + + + + diff --git a/docs/tools/mkdocs-material-theme/partials/flags.html b/docs/tools/mkdocs-material-theme/partials/flags.html index 26d6cdd8f9f..c7b06fbc4d0 100644 --- a/docs/tools/mkdocs-material-theme/partials/flags.html +++ b/docs/tools/mkdocs-material-theme/partials/flags.html @@ -1,4 +1,4 @@ -{% set alt_langs = [['en', 'English'], ['ru', 'Russian'], ['zh', 'Chinese'], ['fa', 'Farsi']] %} +{% set alt_langs = [['en', 'English'], ['ru', 'Russian'], ['zh', 'Chinese'], ['ja', 'Japanese'], ['fa', 'Farsi']] %} {% for alt_lang, alt_title in alt_langs %} > /etc/apt/sources.list +sudo apt-get install clang-8 +``` + +# 安装交叉编译工具集 + +我们假设安装 `cctools` 在 ${CCTOOLS} 路径下 + +```bash +mkdir ${CCTOOLS} + +git clone https://github.com/tpoechtrager/apple-libtapi.git +cd apple-libtapi +INSTALLPREFIX=${CCTOOLS} ./build.sh +./install.sh +cd .. + +git clone https://github.com/tpoechtrager/cctools-port.git +cd cctools-port/cctools +./configure --prefix=${CCTOOLS} --with-libtapi=${CCTOOLS} --target=x86_64-apple-darwin +make install + +cd ${CCTOOLS} +wget https://github.com/phracker/MacOSX-SDKs/releases/download/10.14-beta4/MacOSX10.14.sdk.tar.xz +tar xJf MacOSX10.14.sdk.tar.xz +``` + +# 编译 ClickHouse + +```bash +cd ClickHouse +mkdir build-osx +CC=clang-8 CXX=clang++-8 cmake . -Bbuild-osx -DCMAKE_SYSTEM_NAME=Darwin \ + -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar \ + -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib \ + -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld \ + -DSDK_PATH=${CCTOOLS}/MacOSX10.14.sdk +ninja -C build-osx +``` + +生成的二进制文件将具有Mach-O可执行格式,并且不能在Linux上运行。 \ No newline at end of file diff --git a/docs/zh/development/contrib.md b/docs/zh/development/contrib.md deleted file mode 120000 index 4749f95f9ef..00000000000 --- a/docs/zh/development/contrib.md +++ /dev/null @@ -1 +0,0 @@ -../../en/development/contrib.md \ No newline at end of file diff --git a/docs/zh/development/contrib.md b/docs/zh/development/contrib.md new file mode 100644 index 00000000000..6c5535b0d4b --- /dev/null +++ b/docs/zh/development/contrib.md @@ -0,0 +1,34 @@ +# 使用的三方库 + +| Library | License | +| ------- | ------- | +| base64 | [BSD 2-Clause License](https://github.com/aklomp/base64/blob/a27c565d1b6c676beaf297fe503c4518185666f7/LICENSE) | +| boost | [Boost Software License 1.0](https://github.com/ClickHouse-Extras/boost-extra/blob/6883b40449f378019aec792f9983ce3afc7ff16e/LICENSE_1_0.txt) | +| brotli | [MIT](https://github.com/google/brotli/blob/master/LICENSE) | +| capnproto | [MIT](https://github.com/capnproto/capnproto/blob/master/LICENSE) | +| cctz | [Apache License 2.0](https://github.com/google/cctz/blob/4f9776a310f4952454636363def82c2bf6641d5f/LICENSE.txt) | +| double-conversion | [BSD 3-Clause License](https://github.com/google/double-conversion/blob/cf2f0f3d547dc73b4612028a155b80536902ba02/LICENSE) | +| FastMemcpy | [MIT](https://github.com/yandex/ClickHouse/blob/master/libs/libmemcpy/impl/LICENSE) | +| googletest | [BSD 3-Clause License](https://github.com/google/googletest/blob/master/LICENSE) | +| hyperscan | [BSD 3-Clause License](https://github.com/intel/hyperscan/blob/master/LICENSE) | +| libbtrie | [BSD 2-Clause License](https://github.com/yandex/ClickHouse/blob/master/contrib/libbtrie/LICENSE) | +| libcxxabi | [BSD + MIT](https://github.com/yandex/ClickHouse/blob/master/libs/libglibc-compatibility/libcxxabi/LICENSE.TXT) | +| libdivide | [Zlib License](https://github.com/yandex/ClickHouse/blob/master/contrib/libdivide/LICENSE.txt) | +| libgsasl | [LGPL v2.1](https://github.com/ClickHouse-Extras/libgsasl/blob/3b8948a4042e34fb00b4fb987535dc9e02e39040/LICENSE) +| libhdfs3 | [Apache License 2.0](https://github.com/ClickHouse-Extras/libhdfs3/blob/bd6505cbb0c130b0db695305b9a38546fa880e5a/LICENSE.txt) | +| libmetrohash | [Apache License 2.0](https://github.com/yandex/ClickHouse/blob/master/contrib/libmetrohash/LICENSE) | +| libpcg-random | [Apache License 2.0](https://github.com/yandex/ClickHouse/blob/master/contrib/libpcg-random/LICENSE-APACHE.txt) | +| libressl | [OpenSSL License](https://github.com/ClickHouse-Extras/ssl/blob/master/COPYING) | +| librdkafka | [BSD 2-Clause License](https://github.com/edenhill/librdkafka/blob/363dcad5a23dc29381cc626620e68ae418b3af19/LICENSE) | +| libwidechar\_width | [CC0 1.0 Universal](https://github.com/yandex/ClickHouse/blob/master/libs/libwidechar_width/LICENSE) | +| llvm | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/llvm/blob/163def217817c90fb982a6daf384744d8472b92b/llvm/LICENSE.TXT) | +| lz4 | [BSD 2-Clause License](https://github.com/lz4/lz4/blob/c10863b98e1503af90616ae99725ecd120265dfb/LICENSE) | +| mariadb-connector-c | [LGPL v2.1](https://github.com/ClickHouse-Extras/mariadb-connector-c/blob/3.1/COPYING.LIB) | +| murmurhash | [Public Domain](https://github.com/yandex/ClickHouse/blob/master/contrib/murmurhash/LICENSE) +| pdqsort | [Zlib License](https://github.com/yandex/ClickHouse/blob/master/contrib/pdqsort/license.txt) | +| poco | [Boost Software License - Version 1.0](https://github.com/ClickHouse-Extras/poco/blob/fe5505e56c27b6ecb0dcbc40c49dc2caf4e9637f/LICENSE) | +| protobuf | [BSD 3-Clause License](https://github.com/ClickHouse-Extras/protobuf/blob/12735370922a35f03999afff478e1c6d7aa917a4/LICENSE) | +| re2 | [BSD 3-Clause License](https://github.com/google/re2/blob/7cf8b88e8f70f97fd4926b56aa87e7f53b2717e0/LICENSE) | +| UnixODBC | [LGPL v2.1](https://github.com/ClickHouse-Extras/UnixODBC/tree/b0ad30f7f6289c12b76f04bfb9d466374bb32168) | +| zlib-ng | [Zlib License](https://github.com/ClickHouse-Extras/zlib-ng/blob/develop/LICENSE.md) | +| zstd | [BSD 3-Clause License](https://github.com/facebook/zstd/blob/dev/LICENSE) | diff --git a/docs/zh/development/developer_instruction.md b/docs/zh/development/developer_instruction.md index 52e1d0fb1e2..cbd9371402d 100644 --- a/docs/zh/development/developer_instruction.md +++ b/docs/zh/development/developer_instruction.md @@ -9,7 +9,7 @@ ClickHose支持Linux,FreeBSD 及 Mac OS X 系统。 您需要(申请)一个GitHub账户来使用ClickHouse。 -如果没有账户,请在https://github.com上注册一个。如果没有SSH密钥,请在本地创建密钥并将其上传到GitHub上。这些交互都是必须的,也可以使用与其他任何SSH服务器相同的密钥。 +如果没有账户,请在https://github.com上注册一个。如果没有SSH密钥,请在本地创建密钥并将公钥上传到GitHub上。这有助于你提交更新代码。并且在不同的SSH服务端,你也可以使用相同的SSH密钥。 要创建ClickHouse源码库的分支,请在https://github.com/ClickHouse/ClickHouse页面上点击右上角的"fork"按钮。它会在本账户上创建您个人的ClickHouse/ClickHouse分支。 diff --git a/docs/zh/development/tests.md b/docs/zh/development/tests.md index 2b5fb7ca0e6..2861697fb0c 100644 --- a/docs/zh/development/tests.md +++ b/docs/zh/development/tests.md @@ -166,7 +166,7 @@ clickhouse benchmark --concurrency 16 < queries.tsv 当我们扩展 ClickHouse 网络协议时,我们手动测试旧的 clickhouse-client 与新的 clickhouse-server 和新的clickhouse-client 一起使用旧的 clickhouse-server (只需从相应的包中运行二进制文件) -## 来自编译器的帮助 +## 来自编译器的提示 ClickHouse 主要的代码 (位于`dbms`目录中) 使用 `-Wall -Wextra -Werror` 构建,并带有一些其他已启用的警告。 虽然没有为第三方库启用这些选项。 diff --git a/docs/zh/operations/monitoring.md b/docs/zh/operations/monitoring.md deleted file mode 120000 index 515ae8b4fff..00000000000 --- a/docs/zh/operations/monitoring.md +++ /dev/null @@ -1 +0,0 @@ -../../en/operations/monitoring.md \ No newline at end of file diff --git a/docs/zh/operations/monitoring.md b/docs/zh/operations/monitoring.md new file mode 100644 index 00000000000..5ad0a1846a2 --- /dev/null +++ b/docs/zh/operations/monitoring.md @@ -0,0 +1,37 @@ +# 监控 + +可以监控到: + +- 硬件资源的利用率。 +- ClickHouse 服务的指标。 + +## 硬件资源利用率 + +ClickHouse 本身不会去监控硬件资源的状态。 + +强烈推荐监控以下监控项: + +- 处理器上的负载和温度。 + + 可以使用 [dmesg](https://en.wikipedia.org/wiki/Dmesg), [turbostat](https://www.linux.org/docs/man8/turbostat.html) 或者其他工具。 + +- 磁盘存储,RAM和网络的使用率。 + +## ClickHouse 服务的指标。 + +ClickHouse服务本身具有用于自我状态监视指标。 + +要跟踪服务器事件,请观察服务器日志。 请参阅配置文件的[logger](server_settings/settings.md#server_settings-logger)部分。 + +ClickHouse 收集的指标项: + +- 服务用于计算的资源占用的各种指标。 +- 关于查询处理的常见统计信息。 + +可以在 [system.metrics](system_tables.md#system_tables-metrics) ,[system.events](system_tables.md#system_tables-events) 以及[system.asynchronous_metrics](system_tables.md#system_tables-asynchronous_metrics) 等系统表查看所有的指标项。 + +可以配置ClickHouse 往 [Graphite](https://github.com/graphite-project)导入指标。 参考 [Graphite section](server_settings/settings.md#server_settings-graphite) 配置文件。在配置指标导出之前,需要参考Graphite[官方教程](https://graphite.readthedocs.io/en/latest/install.html)搭建服务。 + +此外,您可以通过HTTP API监视服务器可用性。 将HTTP GET请求发送到 `/`。 如果服务器可用,它将以 `200 OK` 响应。 + +要监视服务器集群的配置中,应设置[max_replica_delay_for_distributed_queries](settings/settings.md#settings-max_replica_delay_for_distributed_queries)参数并使用HTTP资源`/replicas_status`。 如果副本可用,并且不延迟在其他副本之后,则对`/replicas_status`的请求将返回200 OK。 如果副本被延迟,它将返回有关延迟信息。 diff --git a/docs/zh/roadmap.md b/docs/zh/roadmap.md deleted file mode 120000 index 24df86352b3..00000000000 --- a/docs/zh/roadmap.md +++ /dev/null @@ -1 +0,0 @@ -../en/roadmap.md \ No newline at end of file diff --git a/docs/zh/roadmap.md b/docs/zh/roadmap.md new file mode 100644 index 00000000000..3be2aa01533 --- /dev/null +++ b/docs/zh/roadmap.md @@ -0,0 +1,16 @@ +# 规划 + +## Q3 2019 + +- 字典表的DDL +- 与类S3对象存储集成 +- 冷热数据存储分离,支持JBOD + +## Q4 2019 + +- JOIN 不受可用内存限制 +- 更精确的用户资源池,可以在用户之间合理分配集群资源 +- 细粒度的授权管理 +- 与外部认证服务集成 + +[来源文章](https://clickhouse.yandex/docs/en/roadmap/) diff --git a/docs/zh/security_changelog.md b/docs/zh/security_changelog.md deleted file mode 120000 index 101a4f4e48c..00000000000 --- a/docs/zh/security_changelog.md +++ /dev/null @@ -1 +0,0 @@ -../en/security_changelog.md \ No newline at end of file diff --git a/docs/zh/security_changelog.md b/docs/zh/security_changelog.md new file mode 100644 index 00000000000..f4e5910c6d2 --- /dev/null +++ b/docs/zh/security_changelog.md @@ -0,0 +1,39 @@ +## 修复于 ClickHouse Release 18.12.13, 2018-09-10 + +### CVE-2018-14672 + +加载CatBoost模型的功能,允许遍历路径并通过错误消息读取任意文件。 + +来源: Yandex信息安全团队的Andrey Krasichkov + +## 修复于 ClickHouse Release 18.10.3, 2018-08-13 + +### CVE-2018-14671 + +unixODBC允许从文件系统加载任意共享对象,从而导致“远程执行代码”漏洞。 + +来源:Yandex信息安全团队的Andrey Krasichkov和Evgeny Sidorov + +## 修复于 ClickHouse Release 1.1.54388, 2018-06-28 + +### CVE-2018-14668 +远程表函数功能允许在 "user", "password" 及 "default_database" 字段中使用任意符号,从而导致跨协议请求伪造攻击。 + +来源:Yandex信息安全团队的Andrey Krasichkov + +## 修复于 ClickHouse Release 1.1.54390, 2018-07-06 + +### CVE-2018-14669 +ClickHouse MySQL客户端启用了 "LOAD DATA LOCAL INFILE" 功能,该功能允许恶意MySQL数据库从连接的ClickHouse服务器读取任意文件。 + +来源:Yandex信息安全团队的Andrey Krasichkov和Evgeny Sidorov + +## 修复于 ClickHouse Release 1.1.54131, 2017-01-10 + +### CVE-2018-14670 + +deb软件包中的错误配置可能导致使用未经授权的数据库。 + +来源:英国国家网络安全中心(NCSC) + +[来源文章](https://clickhouse.yandex/docs/en/security_changelog/) diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 931d91bd8b5..15b61c9b454 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -110,7 +110,7 @@ static void faultSignalHandler(int sig, siginfo_t * info, void * context) out.next(); - if (sig != SIGPROF) /// This signal is used for debugging. + if (sig != SIGTSTP) /// This signal is used for debugging. { /// The time that is usually enough for separate thread to print info into log. ::sleep(10); @@ -719,9 +719,9 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() } }; - /// SIGPROF is added for debugging purposes. To output a stack trace of any running thread at anytime. + /// SIGTSTP is added for debugging purposes. To output a stack trace of any running thread at anytime. - add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGPROF}, faultSignalHandler); + add_signal_handler({SIGABRT, SIGSEGV, SIGILL, SIGBUS, SIGSYS, SIGFPE, SIGPIPE, SIGTSTP}, faultSignalHandler); add_signal_handler({SIGHUP, SIGUSR1}, closeLogsSignalHandler); add_signal_handler({SIGINT, SIGQUIT, SIGTERM}, terminateRequestedSignalHandler); @@ -731,7 +731,6 @@ void BaseDaemon::initializeTerminationAndSignalProcessing() signal_listener.reset(new SignalListener(*this)); signal_listener_thread.start(*signal_listener); - } void BaseDaemon::logRevision() const @@ -891,4 +890,3 @@ void BaseDaemon::waitForTerminationRequest() std::unique_lock lock(signal_handler_mutex); signal_event.wait(lock, [this](){ return terminate_signals_counter > 0; }); } - diff --git a/website/robots.txt b/website/robots.txt index db843cdbf06..82708ceea95 100644 --- a/website/robots.txt +++ b/website/robots.txt @@ -2,16 +2,16 @@ User-agent: * Disallow: /docs/en/single/ Disallow: /docs/ru/single/ Disallow: /docs/zh/single/ +Disallow: /docs/ja/single/ Disallow: /docs/fa/single/ Disallow: /docs/v1* Disallow: /docs/v2* Disallow: /docs/v3* Disallow: /docs/en/search.html Disallow: /docs/ru/search.html -Disallow: /docs/fa/search.html +Disallow: /docs/ja/search.html Disallow: /docs/zh/search.html -Disallow: /deprecated/reference_en.html -Disallow: /deprecated/reference_ru.html +Disallow: /docs/fa/search.html Allow: / Host: https://clickhouse.yandex Sitemap: https://clickhouse.yandex/docs/sitemap.xml diff --git a/website/sitemap.xml b/website/sitemap.xml index db7bd695b58..e9319dc8701 100644 --- a/website/sitemap.xml +++ b/website/sitemap.xml @@ -9,6 +9,9 @@ https://clickhouse.yandex/docs/zh/sitemap.xml + + https://clickhouse.yandex/docs/ja/sitemap.xml + https://clickhouse.yandex/docs/fa/sitemap.xml