From e1f345cc6b91b53553b5695a81557470bc85dd10 Mon Sep 17 00:00:00 2001 From: Joanna Hulboj Date: Sat, 4 Feb 2023 11:16:18 +0000 Subject: [PATCH 01/14] Improve behaviour of conversion into Date for boundary value 65535 - with fixed tests --- src/Functions/FunctionsConversion.h | 4 +- ...lumn_consistent_insert_behaviour.reference | 36 +++++++ ...ate_column_consistent_insert_behaviour.sql | 99 +++++++++++++++++++ 3 files changed, 137 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference create mode 100644 tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index c34cd3ac875..55003044ff5 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -377,7 +377,7 @@ struct ToDateTransform32Or64 static NO_SANITIZE_UNDEFINED ToType execute(const FromType & from, const DateLUTImpl & time_zone) { // since converting to Date, no need in values outside of default LUT range. - return (from < DATE_LUT_MAX_DAY_NUM) + return (from <= DATE_LUT_MAX_DAY_NUM) ? from : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } @@ -394,7 +394,7 @@ struct ToDateTransform32Or64Signed /// The function should be monotonic (better for query optimizations), so we saturate instead of overflow. if (from < 0) return 0; - return (from < DATE_LUT_MAX_DAY_NUM) + return (from <= DATE_LUT_MAX_DAY_NUM) ? static_cast(from) : time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))); } diff --git a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference new file mode 100644 index 00000000000..5059130c000 --- /dev/null +++ b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.reference @@ -0,0 +1,36 @@ +2149-06-06 65535 +2149-06-06 toUInt16(65535) +2149-06-06 toInt32(65535) +2149-06-06 toUInt32(65535) +2149-06-06 toDate(65535) +2149-06-06 CAST(65535 as UInt16) +2149-06-06 CAST(65535 as Int32) +2149-06-06 CAST(65535 as UInt32) +2149-06-06 CAST(65535 as Date) +2149-06-05 65534 +2149-06-05 toUInt16(65534) +2149-06-05 toInt32(65534) +2149-06-05 toUInt32(65534) +2149-06-05 toDate(65534) +2149-06-05 CAST(65534 as UInt16) +2149-06-05 CAST(65534 as Int32) +2149-06-05 CAST(65534 as UInt32) +2149-06-05 CAST(65534 as Date) +1970-01-01 0 +1970-01-01 toUInt16(0) +1970-01-01 toInt32(0) +1970-01-01 toUInt32(0) +1970-01-01 toDate(0) +1970-01-01 CAST(0 as UInt16) +1970-01-01 CAST(0 as Int32) +1970-01-01 CAST(0 as UInt32) +1970-01-01 CAST(0 as Date) +1 65536 +1 toInt32(65536) +1 toUInt32(65536) +1 toDate(65536) +1 CAST(65536 as Int32) +1 CAST(65536 as UInt32) +1 CAST(65536 as Date) +1970-01-01 toUInt16(65536) +1970-01-01 CAST(65536 as UInt16) diff --git a/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql new file mode 100644 index 00000000000..8a5c88fcae4 --- /dev/null +++ b/tests/queries/0_stateless/02540_date_column_consistent_insert_behaviour.sql @@ -0,0 +1,99 @@ +DROP TABLE IF EXISTS 02540_date; +CREATE TABLE 02540_date (txt String, x Date) engine=Memory; + +-- Date: Supported range of values: [1970-01-01, 2149-06-06]. +-- ^----closed interval---^ + +INSERT INTO 02540_date VALUES('65535', 65535); +INSERT INTO 02540_date VALUES('toUInt16(65535)', toUInt16(65535)); -- #43370 weird one -> used to be 1970-01-01 +INSERT INTO 02540_date VALUES('toInt32(65535)', toInt32(65535)); +INSERT INTO 02540_date VALUES('toUInt32(65535)', toUInt32(65535)); +INSERT INTO 02540_date VALUES('toDate(65535)', toDate(65535)); + +INSERT INTO 02540_date VALUES('CAST(65535 as UInt16)', CAST(65535 as UInt16)); +INSERT INTO 02540_date VALUES('CAST(65535 as Int32)', CAST(65535 as Int32)); +INSERT INTO 02540_date VALUES('CAST(65535 as UInt32)', CAST(65535 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(65535 as Date)', CAST(65535 as Date)); + +INSERT INTO 02540_date VALUES('65534', 65534); +INSERT INTO 02540_date VALUES('toUInt16(65534)', toUInt16(65534)); +INSERT INTO 02540_date VALUES('toInt32(65534)', toInt32(65534)); +INSERT INTO 02540_date VALUES('toUInt32(65534)', toUInt32(65534)); +INSERT INTO 02540_date VALUES('toDate(65534)', toDate(65534)); + +INSERT INTO 02540_date VALUES('CAST(65534 as UInt16)', CAST(65534 as UInt16)); +INSERT INTO 02540_date VALUES('CAST(65534 as Int32)', CAST(65534 as Int32)); +INSERT INTO 02540_date VALUES('CAST(65534 as UInt32)', CAST(65534 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(65534 as Date)', CAST(65534 as Date)); + +INSERT INTO 02540_date VALUES('0', 0); +INSERT INTO 02540_date VALUES('toUInt16(0)', toUInt16(0)); +INSERT INTO 02540_date VALUES('toInt32(0)', toInt32(0)); +INSERT INTO 02540_date VALUES('toUInt32(0)', toUInt32(0)); +INSERT INTO 02540_date VALUES('toDate(0)', toDate(0)); + +INSERT INTO 02540_date VALUES('CAST(0 as UInt16)', CAST(0 as UInt16)); +INSERT INTO 02540_date VALUES('CAST(0 as Int32)', CAST(0 as Int32)); +INSERT INTO 02540_date VALUES('CAST(0 as UInt32)', CAST(0 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(0 as Date)', CAST(0 as Date)); + + +-- 65536 will be done using the TZ settings (comments in #45914) +-- We can expect either 1970-01-01 or 1970-01-02 +-- time_zone.toDayNum(std::min(time_t(from), time_t(0xFFFFFFFF))) +INSERT INTO 02540_date VALUES('65536', 65536); +INSERT INTO 02540_date VALUES('toUInt16(65536)', toUInt16(65536)); -- Narrowing conversion 65536 ==> 0 +INSERT INTO 02540_date VALUES('toInt32(65536)', toInt32(65536)); +INSERT INTO 02540_date VALUES('toUInt32(65536)', toUInt32(65536)); +INSERT INTO 02540_date VALUES('toDate(65536)', toDate(65536)); + +INSERT INTO 02540_date VALUES('CAST(65536 as UInt16)', CAST(65536 as UInt16)); -- Narrowing conversion 65536 ==> 0 +INSERT INTO 02540_date VALUES('CAST(65536 as Int32)', CAST(65536 as Int32)); +INSERT INTO 02540_date VALUES('CAST(65536 as UInt32)', CAST(65536 as UInt32)); +INSERT INTO 02540_date VALUES('CAST(65536 as Date)', CAST(65536 as Date)); + + +SELECT x, txt FROM 02540_date WHERE txt == '65535'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65535)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(65535)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(65535)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toDate(65535)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as UInt16)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as Int32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as UInt32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65535 as Date)'; + +SELECT x, txt FROM 02540_date WHERE txt == '65534'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65534)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(65534)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(65534)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toDate(65534)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as UInt16)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as Int32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as UInt32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65534 as Date)'; + +SELECT x, txt FROM 02540_date WHERE txt == '0'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(0)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toInt32(0)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt32(0)'; +SELECT x, txt FROM 02540_date WHERE txt == 'toDate(0)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as UInt16)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as Int32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as UInt32)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(0 as Date)'; + +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == '65536'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'toInt32(65536)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'toUInt32(65536)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'toDate(65536)'; + +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'CAST(65536 as Int32)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'CAST(65536 as UInt32)'; +SELECT (x == CAST(65536 as Date)), txt FROM 02540_date WHERE txt == 'CAST(65536 as Date)'; + +SELECT x, txt FROM 02540_date WHERE txt == 'toUInt16(65536)'; +SELECT x, txt FROM 02540_date WHERE txt == 'CAST(65536 as UInt16)'; From 84b9ff450f4ca15a83e2b18c5c5797367a7b14b0 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Fri, 3 Feb 2023 10:54:49 +0000 Subject: [PATCH 02/14] Fix terribly broken, fragile and potentially cyclic linking Sorry for the clickbaity title. This is about static method ConnectionTimeouts::getHTTPTimeouts(). It was be declared in header IO/ConnectionTimeouts.h, and defined in header IO/ConnectionTimeoutsContext.h (!). This is weird and caused issues with linking on s390x (##45520). There was an attempt to fix some inconsistencies (#45848) but neither did @Algunenano nor me at first really understand why the definition is in the header. Turns out that ConnectionTimeoutsContext.h is only #include'd from source files which are part of the normal server build BUT NOT part of the keeper standalone build (which must be enabled via CMake -DBUILD_STANDALONE_KEEPER=1). This dependency was not documented and as a result, some misguided workarounds were introduced earlier, e.g. https://github.com/ClickHouse/ClickHouse/pull/38475/commits/0341c6c54bd7ac77200b4ca123208b195514ef20 The deeper cause was that getHTTPTimeouts() is passed a "Context". This class is part of the "dbms" libary which is deliberately not linked by the standalone build of clickhouse-keeper. The context is only used to read the settings and the "Settings" class is part of the clickhouse_common library which is linked by clickhouse-keeper already. To resolve this mess, this PR - creates source file IO/ConnectionTimeouts.cpp and moves all ConnectionTimeouts definitions into it, including getHTTPTimeouts(). - breaks the wrong dependency by passing "Settings" instead of "Context" into getHTTPTimeouts(). - resolves the previous hacks --- programs/benchmark/Benchmark.cpp | 1 - programs/copier/ClusterCopierApp.cpp | 2 +- src/BridgeHelper/LibraryBridgeHelper.cpp | 6 +- src/BridgeHelper/XDBCBridgeHelper.h | 23 +++- src/CMakeLists.txt | 5 +- src/Compression/CompressionFactory.cpp | 2 - src/Core/SettingsFields.cpp | 11 +- src/Core/SettingsFields.h | 4 - src/Dictionaries/HTTPDictionarySource.cpp | 13 +- src/Dictionaries/XDBCDictionarySource.cpp | 8 +- .../ObjectStorages/Web/WebObjectStorage.cpp | 8 +- src/IO/ConnectionTimeouts.cpp | 123 ++++++++++++++++++ src/IO/ConnectionTimeouts.h | 75 ++--------- src/IO/ConnectionTimeoutsContext.h | 38 ------ .../ClusterProxy/SelectStreamFactory.cpp | 2 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 1 - src/QueryPipeline/RemoteQueryExecutor.cpp | 2 +- src/Storages/Distributed/DirectoryMonitor.cpp | 1 - src/Storages/Distributed/DistributedSink.cpp | 2 +- src/Storages/HDFS/StorageHDFSCluster.cpp | 2 +- src/Storages/StorageDistributed.cpp | 2 +- src/Storages/StorageReplicatedMergeTree.cpp | 17 ++- src/Storages/StorageS3Cluster.cpp | 2 +- src/Storages/StorageURL.cpp | 40 ++++-- src/Storages/StorageXDBC.cpp | 9 +- src/TableFunctions/ITableFunctionXDBC.cpp | 9 +- 26 files changed, 256 insertions(+), 152 deletions(-) create mode 100644 src/IO/ConnectionTimeouts.cpp delete mode 100644 src/IO/ConnectionTimeoutsContext.h diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp index 7fb0b1f154f..994f9b7ac4d 100644 --- a/programs/benchmark/Benchmark.cpp +++ b/programs/benchmark/Benchmark.cpp @@ -26,7 +26,6 @@ #include #include #include -#include #include #include #include diff --git a/programs/copier/ClusterCopierApp.cpp b/programs/copier/ClusterCopierApp.cpp index bd505b319bb..b2994b90e23 100644 --- a/programs/copier/ClusterCopierApp.cpp +++ b/programs/copier/ClusterCopierApp.cpp @@ -1,7 +1,7 @@ #include "ClusterCopierApp.h" #include #include -#include +#include #include #include #include diff --git a/src/BridgeHelper/LibraryBridgeHelper.cpp b/src/BridgeHelper/LibraryBridgeHelper.cpp index 6545c3c3b42..f5fd261e046 100644 --- a/src/BridgeHelper/LibraryBridgeHelper.cpp +++ b/src/BridgeHelper/LibraryBridgeHelper.cpp @@ -1,6 +1,6 @@ #include "LibraryBridgeHelper.h" -#include +#include namespace DB { @@ -12,8 +12,10 @@ LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_) , http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value) , bridge_host(config.getString("library_bridge.host", DEFAULT_HOST)) , bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT)) - , http_timeouts(ConnectionTimeouts::getHTTPTimeouts(context_)) { + const auto & settings = context_->getSettingsRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + http_timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); } diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index e3806b85ca9..e0dec98410f 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -12,7 +12,7 @@ #include #include #include -#include +#include #include #include @@ -96,9 +96,14 @@ protected: bool bridgeHandShake() override { + const auto & settings = getContext()->getSettingsRef(); + const auto & cfg = getContext()->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{cfg.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, ConnectionTimeouts::getHTTPTimeouts(getContext()), credentials); + ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, timeouts, credentials); return checkString(PING_OK_ANSWER, buf); } catch (...) @@ -195,7 +200,12 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(getContext()), credentials); + const auto & settings = getContext()->getSettingsRef(); + const auto & cfg = getContext()->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{cfg.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + + ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts, credentials); bool res; readBoolText(res, buf); @@ -217,7 +227,12 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(getContext()), credentials); + const auto & settings = getContext()->getSettingsRef(); + const auto & cfg = getContext()->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{cfg.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + + ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts, credentials); std::string character; readStringBinary(character, buf); diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 82958394224..28bdd846597 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -337,14 +337,13 @@ set_source_files_properties( PROPERTIES COMPILE_FLAGS "-mwaitpkg") endif () -target_link_libraries(common PUBLIC ch_contrib::re2_st) -target_link_libraries(common PUBLIC ch_contrib::re2) - target_link_libraries(clickhouse_common_io PUBLIC boost::program_options boost::system ch_contrib::cityhash + ch_contrib::re2 + ch_contrib::re2_st ch_contrib::zlib pcg_random Poco::Foundation diff --git a/src/Compression/CompressionFactory.cpp b/src/Compression/CompressionFactory.cpp index e4be73c418c..ca08da0ab39 100644 --- a/src/Compression/CompressionFactory.cpp +++ b/src/Compression/CompressionFactory.cpp @@ -173,14 +173,12 @@ void registerCodecDeflateQpl(CompressionCodecFactory & factory); /// Keeper use only general-purpose codecs, so we don't need these special codecs /// in standalone build #ifndef KEEPER_STANDALONE_BUILD - void registerCodecDelta(CompressionCodecFactory & factory); void registerCodecT64(CompressionCodecFactory & factory); void registerCodecDoubleDelta(CompressionCodecFactory & factory); void registerCodecGorilla(CompressionCodecFactory & factory); void registerCodecEncrypted(CompressionCodecFactory & factory); void registerCodecFPC(CompressionCodecFactory & factory); - #endif CompressionCodecFactory::CompressionCodecFactory() diff --git a/src/Core/SettingsFields.cpp b/src/Core/SettingsFields.cpp index c85fb073b8c..4164bf1e27e 100644 --- a/src/Core/SettingsFields.cpp +++ b/src/Core/SettingsFields.cpp @@ -54,7 +54,6 @@ namespace return applyVisitor(FieldVisitorConvertToNumber(), f); } -#ifndef KEEPER_STANDALONE_BUILD Map stringToMap(const String & str) { /// Allow empty string as an empty map @@ -71,7 +70,7 @@ namespace return (*column)[0].safeGet(); } - Map fieldToMap(const Field & f) + [[maybe_unused]] Map fieldToMap(const Field & f) { if (f.getType() == Field::Types::String) { @@ -82,7 +81,6 @@ namespace return f.safeGet(); } -#endif } @@ -327,6 +325,13 @@ void SettingFieldString::readBinary(ReadBuffer & in) *this = std::move(str); } +/// Unbeautiful workaround for clickhouse-keeper standalone build ("-DBUILD_STANDALONE_KEEPER=1"). +/// In this build, we don't build and link library dbms (to which SettingsField.cpp belongs) but +/// only build SettingsField.cpp. Further dependencies, e.g. DataTypeString and DataTypeMap below, +/// require building of further files for clickhouse-keeper. To keep dependencies slim, we don't do +/// that. The linker does not complain only because clickhouse-keeper does not call any of below +/// functions. A cleaner alternative would be more modular libraries, e.g. one for data types, which +/// could then be linked by the server and the linker. #ifndef KEEPER_STANDALONE_BUILD SettingFieldMap::SettingFieldMap(const Field & f) : value(fieldToMap(f)) {} diff --git a/src/Core/SettingsFields.h b/src/Core/SettingsFields.h index 4f339380f0e..c6fe46c9f6b 100644 --- a/src/Core/SettingsFields.h +++ b/src/Core/SettingsFields.h @@ -239,8 +239,6 @@ struct SettingFieldString void readBinary(ReadBuffer & in); }; -#ifndef KEEPER_STANDALONE_BUILD - struct SettingFieldMap { public: @@ -264,8 +262,6 @@ public: void readBinary(ReadBuffer & in); }; -#endif - struct SettingFieldChar { public: diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 2961c3118cd..bfd74228c13 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -1,7 +1,6 @@ #include "HTTPDictionarySource.h" #include #include -#include #include #include #include @@ -39,8 +38,12 @@ HTTPDictionarySource::HTTPDictionarySource( , configuration(configuration_) , sample_block(sample_block_) , context(context_) - , timeouts(ConnectionTimeouts::getHTTPTimeouts(context)) { + const auto & settings = context->getSettingsRef(); + const auto & config = context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + credentials.setUsername(credentials_.getUsername()); credentials.setPassword(credentials_.getPassword()); } @@ -52,8 +55,12 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) , configuration(other.configuration) , sample_block(other.sample_block) , context(Context::createCopy(other.context)) - , timeouts(ConnectionTimeouts::getHTTPTimeouts(context)) { + const auto & settings = context->getSettingsRef(); + const auto & config = context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + credentials.setUsername(other.credentials.getUsername()); credentials.setPassword(other.credentials.getPassword()); } diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 7ff8a8ece15..02796811f4d 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -4,7 +4,7 @@ #include #include #include -#include +#include #include #include #include @@ -76,8 +76,12 @@ XDBCDictionarySource::XDBCDictionarySource( , load_all_query(query_builder.composeLoadAllQuery()) , bridge_helper(bridge_) , bridge_url(bridge_helper->getMainURI()) - , timeouts(ConnectionTimeouts::getHTTPTimeouts(context_)) { + const auto & settings = context_->getSettingsRef(); + const auto & config = context_->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + auto url_params = bridge_helper->getURLParams(max_block_size); for (const auto & [name, value] : url_params) bridge_url.addQueryParameter(name, value); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index ba39150e164..e6b8e5c976e 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -40,11 +40,17 @@ void WebObjectStorage::initialize(const String & uri_path) const try { Poco::Net::HTTPBasicCredentials credentials{}; + + const auto & settings = getContext()->getSettingsRef(); + const auto & config = getContext()->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + ReadWriteBufferFromHTTP metadata_buf( Poco::URI(fs::path(uri_path) / ".index"), Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), - ConnectionTimeouts::getHTTPTimeouts(getContext()), + timeouts, credentials, /* max_redirects= */ 0, /* buffer_size_= */ DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp new file mode 100644 index 00000000000..4e3d7a2f5f5 --- /dev/null +++ b/src/IO/ConnectionTimeouts.cpp @@ -0,0 +1,123 @@ +#include +#include +#include + +namespace DB +{ + +ConnectionTimeouts::ConnectionTimeouts( + Poco::Timespan connection_timeout_, + Poco::Timespan send_timeout_, + Poco::Timespan receive_timeout_) + : connection_timeout(connection_timeout_) + , send_timeout(send_timeout_) + , receive_timeout(receive_timeout_) + , tcp_keep_alive_timeout(0) + , http_keep_alive_timeout(0) + , secure_connection_timeout(connection_timeout) + , hedged_connection_timeout(receive_timeout_) + , receive_data_timeout(receive_timeout_) +{ +} + +ConnectionTimeouts::ConnectionTimeouts( + Poco::Timespan connection_timeout_, + Poco::Timespan send_timeout_, + Poco::Timespan receive_timeout_, + Poco::Timespan tcp_keep_alive_timeout_) + : connection_timeout(connection_timeout_) + , send_timeout(send_timeout_) + , receive_timeout(receive_timeout_) + , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) + , http_keep_alive_timeout(0) + , secure_connection_timeout(connection_timeout) + , hedged_connection_timeout(receive_timeout_) + , receive_data_timeout(receive_timeout_) +{ +} + +ConnectionTimeouts::ConnectionTimeouts( + Poco::Timespan connection_timeout_, + Poco::Timespan send_timeout_, + Poco::Timespan receive_timeout_, + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan http_keep_alive_timeout_) + : connection_timeout(connection_timeout_) + , send_timeout(send_timeout_) + , receive_timeout(receive_timeout_) + , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) + , http_keep_alive_timeout(http_keep_alive_timeout_) + , secure_connection_timeout(connection_timeout) + , hedged_connection_timeout(receive_timeout_) + , receive_data_timeout(receive_timeout_) +{ +} + +ConnectionTimeouts::ConnectionTimeouts( + Poco::Timespan connection_timeout_, + Poco::Timespan send_timeout_, + Poco::Timespan receive_timeout_, + Poco::Timespan tcp_keep_alive_timeout_, + Poco::Timespan http_keep_alive_timeout_, + Poco::Timespan secure_connection_timeout_, + Poco::Timespan receive_hello_timeout_, + Poco::Timespan receive_data_timeout_) + : connection_timeout(connection_timeout_) + , send_timeout(send_timeout_) + , receive_timeout(receive_timeout_) + , tcp_keep_alive_timeout(tcp_keep_alive_timeout_) + , http_keep_alive_timeout(http_keep_alive_timeout_) + , secure_connection_timeout(secure_connection_timeout_) + , hedged_connection_timeout(receive_hello_timeout_) + , receive_data_timeout(receive_data_timeout_) +{ +} + +Poco::Timespan ConnectionTimeouts::saturate(Poco::Timespan timespan, Poco::Timespan limit) +{ + if (limit.totalMicroseconds() == 0) + return timespan; + else + return (timespan > limit) ? limit : timespan; +} + +ConnectionTimeouts ConnectionTimeouts::getSaturated(Poco::Timespan limit) const +{ + return ConnectionTimeouts(saturate(connection_timeout, limit), + saturate(send_timeout, limit), + saturate(receive_timeout, limit), + saturate(tcp_keep_alive_timeout, limit), + saturate(http_keep_alive_timeout, limit), + saturate(secure_connection_timeout, limit), + saturate(hedged_connection_timeout, limit), + saturate(receive_data_timeout, limit)); +} + +/// Timeouts for the case when we have just single attempt to connect. +ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(const Settings & settings) +{ + return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout); +} + +/// Timeouts for the case when we will try many addresses in a loop. +ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings & settings) +{ + return ConnectionTimeouts( + settings.connect_timeout_with_failover_ms, + settings.send_timeout, + settings.receive_timeout, + settings.tcp_keep_alive_timeout, + 0, + settings.connect_timeout_with_failover_secure_ms, + settings.hedged_connection_timeout_ms, + settings.receive_data_timeout_ms); +} + +ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts( + const Settings & settings, + Poco::Timespan http_keep_alive_timeout) +{ + return ConnectionTimeouts(settings.http_connection_timeout, settings.http_send_timeout, settings.http_receive_timeout, settings.tcp_keep_alive_timeout, http_keep_alive_timeout); +} + +} diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 95a2ded6466..298a505bced 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -30,47 +30,18 @@ struct ConnectionTimeouts ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, - Poco::Timespan receive_timeout_) - : connection_timeout(connection_timeout_), - send_timeout(send_timeout_), - receive_timeout(receive_timeout_), - tcp_keep_alive_timeout(0), - http_keep_alive_timeout(0), - secure_connection_timeout(connection_timeout), - hedged_connection_timeout(receive_timeout_), - receive_data_timeout(receive_timeout_) - { - } + Poco::Timespan receive_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, - Poco::Timespan tcp_keep_alive_timeout_) - : connection_timeout(connection_timeout_), - send_timeout(send_timeout_), - receive_timeout(receive_timeout_), - tcp_keep_alive_timeout(tcp_keep_alive_timeout_), - http_keep_alive_timeout(0), - secure_connection_timeout(connection_timeout), - hedged_connection_timeout(receive_timeout_), - receive_data_timeout(receive_timeout_) - { - } + Poco::Timespan tcp_keep_alive_timeout_); + ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, Poco::Timespan receive_timeout_, Poco::Timespan tcp_keep_alive_timeout_, - Poco::Timespan http_keep_alive_timeout_) - : connection_timeout(connection_timeout_), - send_timeout(send_timeout_), - receive_timeout(receive_timeout_), - tcp_keep_alive_timeout(tcp_keep_alive_timeout_), - http_keep_alive_timeout(http_keep_alive_timeout_), - secure_connection_timeout(connection_timeout), - hedged_connection_timeout(receive_timeout_), - receive_data_timeout(receive_timeout_) - { - } + Poco::Timespan http_keep_alive_timeout_); ConnectionTimeouts(Poco::Timespan connection_timeout_, Poco::Timespan send_timeout_, @@ -79,43 +50,19 @@ struct ConnectionTimeouts Poco::Timespan http_keep_alive_timeout_, Poco::Timespan secure_connection_timeout_, Poco::Timespan receive_hello_timeout_, - Poco::Timespan receive_data_timeout_) - : connection_timeout(connection_timeout_), - send_timeout(send_timeout_), - receive_timeout(receive_timeout_), - tcp_keep_alive_timeout(tcp_keep_alive_timeout_), - http_keep_alive_timeout(http_keep_alive_timeout_), - secure_connection_timeout(secure_connection_timeout_), - hedged_connection_timeout(receive_hello_timeout_), - receive_data_timeout(receive_data_timeout_) - { - } + Poco::Timespan receive_data_timeout_); - static Poco::Timespan saturate(Poco::Timespan timespan, Poco::Timespan limit) - { - if (limit.totalMicroseconds() == 0) - return timespan; - else - return (timespan > limit) ? limit : timespan; - } - - ConnectionTimeouts getSaturated(Poco::Timespan limit) const - { - return ConnectionTimeouts(saturate(connection_timeout, limit), - saturate(send_timeout, limit), - saturate(receive_timeout, limit), - saturate(tcp_keep_alive_timeout, limit), - saturate(http_keep_alive_timeout, limit), - saturate(secure_connection_timeout, limit), - saturate(hedged_connection_timeout, limit), - saturate(receive_data_timeout, limit)); - } + static Poco::Timespan saturate(Poco::Timespan timespan, Poco::Timespan limit); + ConnectionTimeouts getSaturated(Poco::Timespan limit) const; /// Timeouts for the case when we have just single attempt to connect. static ConnectionTimeouts getTCPTimeoutsWithoutFailover(const Settings & settings); + /// Timeouts for the case when we will try many addresses in a loop. static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); - static ConnectionTimeouts getHTTPTimeouts(ContextPtr context); + static ConnectionTimeouts getHTTPTimeouts( + const Settings & settings, + Poco::Timespan http_keep_alive_timeout); }; } diff --git a/src/IO/ConnectionTimeoutsContext.h b/src/IO/ConnectionTimeoutsContext.h deleted file mode 100644 index 8d6f96b82a3..00000000000 --- a/src/IO/ConnectionTimeoutsContext.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ - -/// Timeouts for the case when we have just single attempt to connect. -inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(const Settings & settings) -{ - return ConnectionTimeouts(settings.connect_timeout, settings.send_timeout, settings.receive_timeout, settings.tcp_keep_alive_timeout); -} - -/// Timeouts for the case when we will try many addresses in a loop. -inline ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings & settings) -{ - return ConnectionTimeouts( - settings.connect_timeout_with_failover_ms, - settings.send_timeout, - settings.receive_timeout, - settings.tcp_keep_alive_timeout, - 0, - settings.connect_timeout_with_failover_secure_ms, - settings.hedged_connection_timeout_ms, - settings.receive_data_timeout_ms); -} - -inline ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(ContextPtr context) -{ - const auto & settings = context->getSettingsRef(); - const auto & config = context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - return ConnectionTimeouts(settings.http_connection_timeout, settings.http_send_timeout, settings.http_receive_timeout, settings.tcp_keep_alive_timeout, http_keep_alive_timeout); -} - -} diff --git a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index b08ec7e5ab5..30038ae45b9 100644 --- a/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -7,7 +7,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 9e5ecc791dc..6d36146c7e8 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -12,7 +12,6 @@ #include #include #include -#include #include "Common/logger_useful.h" #include #include diff --git a/src/QueryPipeline/RemoteQueryExecutor.cpp b/src/QueryPipeline/RemoteQueryExecutor.cpp index 033907e9e2b..97a3758cc95 100644 --- a/src/QueryPipeline/RemoteQueryExecutor.cpp +++ b/src/QueryPipeline/RemoteQueryExecutor.cpp @@ -21,7 +21,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/Distributed/DirectoryMonitor.cpp b/src/Storages/Distributed/DirectoryMonitor.cpp index eb1d83af851..f59c9193b43 100644 --- a/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/src/Storages/Distributed/DirectoryMonitor.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include diff --git a/src/Storages/Distributed/DistributedSink.cpp b/src/Storages/Distributed/DistributedSink.cpp index 3cf1ef2678f..bac13ea37cf 100644 --- a/src/Storages/Distributed/DistributedSink.cpp +++ b/src/Storages/Distributed/DistributedSink.cpp @@ -13,7 +13,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index f6e6f773d6c..8dbaa0796e9 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -8,7 +8,7 @@ #include #include #include -#include +#include #include #include #include diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index a2c3a10d836..954dddd07f8 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -81,7 +81,7 @@ #include #include #include -#include +#include #include #include diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 16a2f49b8df..1dafdfb74cb 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -69,7 +69,6 @@ #include #include #include -#include #include #include @@ -2385,7 +2384,10 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr auto metadata_snapshot = getInMemoryMetadataPtr(); String source_replica_path = entry.source_shard + "/replicas/" + replica; ReplicatedMergeTreeAddress address(getZooKeeper()->get(source_replica_path + "/host")); - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(getContext()); + const auto & settings = getContext()->getSettingsRef(); + const auto & config = getContext()->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); @@ -3614,7 +3616,11 @@ void StorageReplicatedMergeTree::stopBeingLeader() ConnectionTimeouts StorageReplicatedMergeTree::getFetchPartHTTPTimeouts(ContextPtr local_context) { - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(local_context); + const auto & sts = local_context->getSettingsRef(); + const auto & config = local_context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(sts, http_keep_alive_timeout); + auto settings = getSettings(); if (settings->replicated_fetches_http_connection_timeout.changed) @@ -4261,7 +4267,10 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( std::function get_part; ReplicatedMergeTreeAddress address(zookeeper->get(fs::path(source_replica_path) / "host")); - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(getContext()); + const auto & settings = getContext()->getSettingsRef(); + const auto & config = getContext()->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index e158aff60a0..e74c4455092 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -8,7 +8,7 @@ #include "Client/Connection.h" #include "Core/QueryProcessingStage.h" #include -#include +#include #include #include #include diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 9bcbc9e6f45..244716bd90e 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -13,7 +13,6 @@ #include #include -#include #include #include #include @@ -625,6 +624,11 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (it == urls_to_check.cend()) return nullptr; + const auto & settings = context->getSettingsRef(); + const auto & config = context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + auto buf = StorageURLSource::getFirstAvailableURLReadBuffer( it, urls_to_check.cend(), @@ -632,7 +636,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( {}, Poco::Net::HTTPRequest::HTTP_GET, {}, - ConnectionTimeouts::getHTTPTimeouts(context), + timeouts, compression_method, credentials, headers, @@ -686,6 +690,11 @@ Pipe IStorageURLBase::read( size_t max_download_threads = local_context->getSettingsRef().max_download_threads; + const auto & settings = local_context->getSettingsRef(); + const auto & config = local_context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + if (urlWithGlobs(uri)) { size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; @@ -716,7 +725,7 @@ Pipe IStorageURLBase::read( local_context, columns_description, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(local_context), + timeouts, compression_method, download_threads, headers, @@ -740,7 +749,7 @@ Pipe IStorageURLBase::read( local_context, columns_description, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(local_context), + timeouts, compression_method, max_download_threads, headers, @@ -775,6 +784,12 @@ Pipe StorageURLWithFailover::read( auto uri_info = std::make_shared(); uri_info->uri_list_to_read.emplace_back(uri_options); + + const auto & settings = local_context->getSettingsRef(); + const auto & config = local_context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + auto pipe = Pipe(std::make_shared( uri_info, getReadMethod(), @@ -786,7 +801,7 @@ Pipe StorageURLWithFailover::read( local_context, columns_description, max_block_size, - ConnectionTimeouts::getHTTPTimeouts(local_context), + timeouts, compression_method, local_context->getSettingsRef().max_download_threads, headers, @@ -806,6 +821,11 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; bool is_partitioned_implementation = partition_by_ast && has_wildcards; + const auto & settings = context->getSettingsRef(); + const auto & config = context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + if (is_partitioned_implementation) { return std::make_shared( @@ -815,7 +835,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad format_settings, metadata_snapshot->getSampleBlock(), context, - ConnectionTimeouts::getHTTPTimeouts(context), + timeouts, compression_method, http_method); } @@ -827,7 +847,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad format_settings, metadata_snapshot->getSampleBlock(), context, - ConnectionTimeouts::getHTTPTimeouts(context), + timeouts, compression_method, http_method); } @@ -890,13 +910,17 @@ std::optional IStorageURLBase::getLastModificationTime( { auto settings = context->getSettingsRef(); + const auto & config = context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + try { ReadWriteBufferFromHTTP buf( Poco::URI(url), Poco::Net::HTTPRequest::HTTP_GET, {}, - ConnectionTimeouts::getHTTPTimeouts(context), + timeouts, credentials, settings.max_http_get_redirects, settings.max_read_buffer_size, diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index 5dd21d98a7e..ab8f59b52dc 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -5,7 +5,7 @@ #include #include -#include +#include #include #include #include @@ -130,13 +130,18 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMet request_uri.addQueryParameter("format_name", format_name); request_uri.addQueryParameter("sample_block", metadata_snapshot->getSampleBlock().getNamesAndTypesList().toString()); + const auto & settings = local_context->getSettingsRef(); + const auto & config = local_context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + return std::make_shared( request_uri.toString(), format_name, getFormatSettings(local_context), metadata_snapshot->getSampleBlock(), local_context, - ConnectionTimeouts::getHTTPTimeouts(local_context), + timeouts, compression_method); } diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index 3abda5061df..8ecd3ac6b2c 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include #include #include #include @@ -75,8 +75,13 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex bool use_nulls = context->getSettingsRef().external_table_functions_use_nulls; columns_info_uri.addQueryParameter("external_table_functions_use_nulls", toString(use_nulls)); + const auto & settings = context->getSettingsRef(); + const auto & config = context->getConfigRef(); + Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; + auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + Poco::Net::HTTPBasicCredentials credentials{}; - ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context), credentials); + ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts, credentials); std::string columns_info; readStringBinary(columns_info, buf); From f1b8d1d9d722d09a279b36f2b91b7887c025e7e2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 7 Feb 2023 00:27:12 +0100 Subject: [PATCH 03/14] Improve performance of Decimal conversion when scale does not change --- src/DataTypes/DataTypesDecimal.h | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/DataTypesDecimal.h b/src/DataTypes/DataTypesDecimal.h index 88b3a33a5b8..7a49238b5be 100644 --- a/src/DataTypes/DataTypesDecimal.h +++ b/src/DataTypes/DataTypesDecimal.h @@ -100,7 +100,7 @@ inline UInt32 getDecimalScale(const DataTypeDecimal & data_type) template requires (IsDataTypeDecimal && IsDataTypeDecimal) -inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result) +inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) { using FromFieldType = typename FromDataType::FieldType; using ToFieldType = typename ToDataType::FieldType; @@ -121,8 +121,14 @@ inline ReturnType convertDecimalsImpl(const typename FromDataType::FieldType & v return ReturnType(false); } } + else if (scale_to == scale_from) + { + converted_value = value.value; + } else + { converted_value = value.value / DecimalUtils::scaleMultiplier(scale_from - scale_to); + } if constexpr (sizeof(FromFieldType) > sizeof(ToFieldType)) { @@ -155,7 +161,7 @@ inline typename ToDataType::FieldType convertDecimals(const typename FromDataTyp template requires (IsDataTypeDecimal && IsDataTypeDecimal) -inline bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType& result) +inline bool tryConvertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_from, UInt32 scale_to, typename ToDataType::FieldType & result) { return convertDecimalsImpl(value, scale_from, scale_to, result); } From 783e26d2babe894e243ee8616573a0619218a6ba Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 11:42:31 +0000 Subject: [PATCH 04/14] Cosmetics --- src/IO/ConnectionTimeouts.cpp | 11 +++++++---- src/IO/ConnectionTimeouts.h | 4 +--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/IO/ConnectionTimeouts.cpp b/src/IO/ConnectionTimeouts.cpp index 4e3d7a2f5f5..401afb7baac 100644 --- a/src/IO/ConnectionTimeouts.cpp +++ b/src/IO/ConnectionTimeouts.cpp @@ -113,11 +113,14 @@ ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithFailover(const Settings settings.receive_data_timeout_ms); } -ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts( - const Settings & settings, - Poco::Timespan http_keep_alive_timeout) +ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout) { - return ConnectionTimeouts(settings.http_connection_timeout, settings.http_send_timeout, settings.http_receive_timeout, settings.tcp_keep_alive_timeout, http_keep_alive_timeout); + return ConnectionTimeouts( + settings.http_connection_timeout, + settings.http_send_timeout, + settings.http_receive_timeout, + settings.tcp_keep_alive_timeout, + http_keep_alive_timeout); } } diff --git a/src/IO/ConnectionTimeouts.h b/src/IO/ConnectionTimeouts.h index 298a505bced..368288ee022 100644 --- a/src/IO/ConnectionTimeouts.h +++ b/src/IO/ConnectionTimeouts.h @@ -60,9 +60,7 @@ struct ConnectionTimeouts /// Timeouts for the case when we will try many addresses in a loop. static ConnectionTimeouts getTCPTimeoutsWithFailover(const Settings & settings); - static ConnectionTimeouts getHTTPTimeouts( - const Settings & settings, - Poco::Timespan http_keep_alive_timeout); + static ConnectionTimeouts getHTTPTimeouts(const Settings & settings, Poco::Timespan http_keep_alive_timeout); }; } From 10af0b3e49cb5e36fefdc94388b13fa68a3d47f7 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 7 Feb 2023 12:10:26 +0000 Subject: [PATCH 05/14] Reduce redundancies --- src/BridgeHelper/LibraryBridgeHelper.cpp | 4 +- src/BridgeHelper/XDBCBridgeHelper.h | 25 ++++------- src/Core/Defines.h | 2 + src/Dictionaries/HTTPDictionarySource.cpp | 12 +----- src/Dictionaries/XDBCDictionarySource.cpp | 6 +-- .../ObjectStorages/Web/WebObjectStorage.cpp | 8 ++-- src/Storages/StorageReplicatedMergeTree.cpp | 22 ++++------ src/Storages/StorageURL.cpp | 42 +++++-------------- src/Storages/StorageXDBC.cpp | 8 ++-- src/TableFunctions/ITableFunctionXDBC.cpp | 14 ++++--- 10 files changed, 46 insertions(+), 97 deletions(-) diff --git a/src/BridgeHelper/LibraryBridgeHelper.cpp b/src/BridgeHelper/LibraryBridgeHelper.cpp index f5fd261e046..60588951c32 100644 --- a/src/BridgeHelper/LibraryBridgeHelper.cpp +++ b/src/BridgeHelper/LibraryBridgeHelper.cpp @@ -12,10 +12,8 @@ LibraryBridgeHelper::LibraryBridgeHelper(ContextPtr context_) , http_timeout(context_->getGlobalContext()->getSettingsRef().http_receive_timeout.value) , bridge_host(config.getString("library_bridge.host", DEFAULT_HOST)) , bridge_port(config.getUInt("library_bridge.port", DEFAULT_PORT)) + , http_timeouts(ConnectionTimeouts::getHTTPTimeouts(context_->getSettingsRef(), {context_->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0})) { - const auto & settings = context_->getSettingsRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - http_timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); } diff --git a/src/BridgeHelper/XDBCBridgeHelper.h b/src/BridgeHelper/XDBCBridgeHelper.h index e0dec98410f..00a661a1fc4 100644 --- a/src/BridgeHelper/XDBCBridgeHelper.h +++ b/src/BridgeHelper/XDBCBridgeHelper.h @@ -96,14 +96,9 @@ protected: bool bridgeHandShake() override { - const auto & settings = getContext()->getSettingsRef(); - const auto & cfg = getContext()->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{cfg.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - try { - ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, timeouts, credentials); + ReadWriteBufferFromHTTP buf(getPingURI(), Poco::Net::HTTPRequest::HTTP_GET, {}, getHTTPTimeouts(), credentials); return checkString(PING_OK_ANSWER, buf); } catch (...) @@ -166,6 +161,10 @@ private: Poco::Net::HTTPBasicCredentials credentials{}; + ConnectionTimeouts getHTTPTimeouts() + { + return ConnectionTimeouts::getHTTPTimeouts(getContext()->getSettingsRef(), {getContext()->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}); + } protected: using URLParams = std::vector>; @@ -200,12 +199,7 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - const auto & settings = getContext()->getSettingsRef(); - const auto & cfg = getContext()->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{cfg.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts, credentials); + ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials); bool res; readBoolText(res, buf); @@ -227,12 +221,7 @@ protected: uri.addQueryParameter("connection_string", getConnectionString()); uri.addQueryParameter("use_connection_pooling", toString(use_connection_pooling)); - const auto & settings = getContext()->getSettingsRef(); - const auto & cfg = getContext()->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{cfg.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - - ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts, credentials); + ReadWriteBufferFromHTTP buf(uri, Poco::Net::HTTPRequest::HTTP_POST, {}, getHTTPTimeouts(), credentials); std::string character; readStringBinary(character, buf); diff --git a/src/Core/Defines.h b/src/Core/Defines.h index 33931d5bca1..3fae123fb6b 100644 --- a/src/Core/Defines.h +++ b/src/Core/Defines.h @@ -52,6 +52,8 @@ /// the number is unmotivated #define DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT 15 +#define DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT 10 + #define DBMS_DEFAULT_PATH "/var/lib/clickhouse/" /// Actually, there may be multiple acquisitions of different locks for a given table within one query. diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index bfd74228c13..7e0056c9cae 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -38,12 +38,8 @@ HTTPDictionarySource::HTTPDictionarySource( , configuration(configuration_) , sample_block(sample_block_) , context(context_) + , timeouts(ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0})) { - const auto & settings = context->getSettingsRef(); - const auto & config = context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - credentials.setUsername(credentials_.getUsername()); credentials.setPassword(credentials_.getPassword()); } @@ -55,12 +51,8 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) , configuration(other.configuration) , sample_block(other.sample_block) , context(Context::createCopy(other.context)) + , timeouts(ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0})) { - const auto & settings = context->getSettingsRef(); - const auto & config = context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - credentials.setUsername(other.credentials.getUsername()); credentials.setPassword(other.credentials.getPassword()); } diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 02796811f4d..36dde92699a 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -76,12 +76,8 @@ XDBCDictionarySource::XDBCDictionarySource( , load_all_query(query_builder.composeLoadAllQuery()) , bridge_helper(bridge_) , bridge_url(bridge_helper->getMainURI()) + , timeouts(ConnectionTimeouts::getHTTPTimeouts(context_->getSettingsRef(), {context_->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0})) { - const auto & settings = context_->getSettingsRef(); - const auto & config = context_->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - auto url_params = bridge_helper->getURLParams(max_block_size); for (const auto & [name, value] : url_params) bridge_url.addQueryParameter(name, value); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index e6b8e5c976e..4365156d93d 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -41,16 +41,14 @@ void WebObjectStorage::initialize(const String & uri_path) const { Poco::Net::HTTPBasicCredentials credentials{}; - const auto & settings = getContext()->getSettingsRef(); - const auto & config = getContext()->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); ReadWriteBufferFromHTTP metadata_buf( Poco::URI(fs::path(uri_path) / ".index"), Poco::Net::HTTPRequest::HTTP_GET, ReadWriteBufferFromHTTP::OutStreamCallback(), - timeouts, + ConnectionTimeouts::getHTTPTimeouts( + getContext()->getSettingsRef(), + {getContext()->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}), credentials, /* max_redirects= */ 0, /* buffer_size_= */ DBMS_DEFAULT_BUFFER_SIZE, diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1dafdfb74cb..5922f561c03 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -232,6 +232,11 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeperAndAssertNotReadonl return res; } +static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) +{ + return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}); +} + static MergeTreePartInfo makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(const String & partition_id) { /// NOTE We don't have special log entry type for MOVE PARTITION/ATTACH PARTITION FROM, @@ -2355,7 +2360,6 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) return true; } - void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entry) { auto zookeeper = getZooKeeper(); @@ -2384,10 +2388,7 @@ void StorageReplicatedMergeTree::executeClonePartFromShard(const LogEntry & entr auto metadata_snapshot = getInMemoryMetadataPtr(); String source_replica_path = entry.source_shard + "/replicas/" + replica; ReplicatedMergeTreeAddress address(getZooKeeper()->get(source_replica_path + "/host")); - const auto & settings = getContext()->getSettingsRef(); - const auto & config = getContext()->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + auto timeouts = getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); @@ -3616,11 +3617,7 @@ void StorageReplicatedMergeTree::stopBeingLeader() ConnectionTimeouts StorageReplicatedMergeTree::getFetchPartHTTPTimeouts(ContextPtr local_context) { - const auto & sts = local_context->getSettingsRef(); - const auto & config = local_context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(sts, http_keep_alive_timeout); - + auto timeouts = getHTTPTimeouts(local_context); auto settings = getSettings(); if (settings->replicated_fetches_http_connection_timeout.changed) @@ -4267,10 +4264,7 @@ MutableDataPartStoragePtr StorageReplicatedMergeTree::fetchExistsPart( std::function get_part; ReplicatedMergeTreeAddress address(zookeeper->get(fs::path(source_replica_path) / "host")); - const auto & settings = getContext()->getSettingsRef(); - const auto & config = getContext()->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); + auto timeouts = getHTTPTimeouts(getContext()); auto credentials = getContext()->getInterserverCredentials(); String interserver_scheme = getContext()->getInterserverScheme(); diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 244716bd90e..152dda8f360 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -81,6 +81,10 @@ static bool urlWithGlobs(const String & uri) return (uri.find('{') != std::string::npos && uri.find('}') != std::string::npos) || uri.find('|') != std::string::npos; } +static ConnectionTimeouts getHTTPTimeouts(ContextPtr context) +{ + return ConnectionTimeouts::getHTTPTimeouts(context->getSettingsRef(), {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}); +} IStorageURLBase::IStorageURLBase( const String & uri_, @@ -624,11 +628,6 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( if (it == urls_to_check.cend()) return nullptr; - const auto & settings = context->getSettingsRef(); - const auto & config = context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - auto buf = StorageURLSource::getFirstAvailableURLReadBuffer( it, urls_to_check.cend(), @@ -636,7 +635,7 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData( {}, Poco::Net::HTTPRequest::HTTP_GET, {}, - timeouts, + getHTTPTimeouts(context), compression_method, credentials, headers, @@ -690,11 +689,6 @@ Pipe IStorageURLBase::read( size_t max_download_threads = local_context->getSettingsRef().max_download_threads; - const auto & settings = local_context->getSettingsRef(); - const auto & config = local_context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - if (urlWithGlobs(uri)) { size_t max_addresses = local_context->getSettingsRef().glob_expansion_max_elements; @@ -725,7 +719,7 @@ Pipe IStorageURLBase::read( local_context, columns_description, max_block_size, - timeouts, + getHTTPTimeouts(local_context), compression_method, download_threads, headers, @@ -749,7 +743,7 @@ Pipe IStorageURLBase::read( local_context, columns_description, max_block_size, - timeouts, + getHTTPTimeouts(local_context), compression_method, max_download_threads, headers, @@ -785,11 +779,6 @@ Pipe StorageURLWithFailover::read( auto uri_info = std::make_shared(); uri_info->uri_list_to_read.emplace_back(uri_options); - const auto & settings = local_context->getSettingsRef(); - const auto & config = local_context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - auto pipe = Pipe(std::make_shared( uri_info, getReadMethod(), @@ -801,7 +790,7 @@ Pipe StorageURLWithFailover::read( local_context, columns_description, max_block_size, - timeouts, + getHTTPTimeouts(local_context), compression_method, local_context->getSettingsRef().max_download_threads, headers, @@ -821,11 +810,6 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad auto partition_by_ast = insert_query ? (insert_query->partition_by ? insert_query->partition_by : partition_by) : nullptr; bool is_partitioned_implementation = partition_by_ast && has_wildcards; - const auto & settings = context->getSettingsRef(); - const auto & config = context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - if (is_partitioned_implementation) { return std::make_shared( @@ -835,7 +819,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad format_settings, metadata_snapshot->getSampleBlock(), context, - timeouts, + getHTTPTimeouts(context), compression_method, http_method); } @@ -847,7 +831,7 @@ SinkToStoragePtr IStorageURLBase::write(const ASTPtr & query, const StorageMetad format_settings, metadata_snapshot->getSampleBlock(), context, - timeouts, + getHTTPTimeouts(context), compression_method, http_method); } @@ -910,17 +894,13 @@ std::optional IStorageURLBase::getLastModificationTime( { auto settings = context->getSettingsRef(); - const auto & config = context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - try { ReadWriteBufferFromHTTP buf( Poco::URI(url), Poco::Net::HTTPRequest::HTTP_GET, {}, - timeouts, + getHTTPTimeouts(context), credentials, settings.max_http_get_redirects, settings.max_read_buffer_size, diff --git a/src/Storages/StorageXDBC.cpp b/src/Storages/StorageXDBC.cpp index ab8f59b52dc..cb5532e91ac 100644 --- a/src/Storages/StorageXDBC.cpp +++ b/src/Storages/StorageXDBC.cpp @@ -130,10 +130,6 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMet request_uri.addQueryParameter("format_name", format_name); request_uri.addQueryParameter("sample_block", metadata_snapshot->getSampleBlock().getNamesAndTypesList().toString()); - const auto & settings = local_context->getSettingsRef(); - const auto & config = local_context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); return std::make_shared( request_uri.toString(), @@ -141,7 +137,9 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMet getFormatSettings(local_context), metadata_snapshot->getSampleBlock(), local_context, - timeouts, + ConnectionTimeouts::getHTTPTimeouts( + local_context->getSettingsRef(), + {local_context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}), compression_method); } diff --git a/src/TableFunctions/ITableFunctionXDBC.cpp b/src/TableFunctions/ITableFunctionXDBC.cpp index 8ecd3ac6b2c..1fb0f392e33 100644 --- a/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/src/TableFunctions/ITableFunctionXDBC.cpp @@ -75,13 +75,15 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex bool use_nulls = context->getSettingsRef().external_table_functions_use_nulls; columns_info_uri.addQueryParameter("external_table_functions_use_nulls", toString(use_nulls)); - const auto & settings = context->getSettingsRef(); - const auto & config = context->getConfigRef(); - Poco::Timespan http_keep_alive_timeout{config.getUInt("keep_alive_timeout", 10), 0}; - auto timeouts = ConnectionTimeouts::getHTTPTimeouts(settings, http_keep_alive_timeout); - Poco::Net::HTTPBasicCredentials credentials{}; - ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, {}, timeouts, credentials); + ReadWriteBufferFromHTTP buf( + columns_info_uri, + Poco::Net::HTTPRequest::HTTP_POST, + {}, + ConnectionTimeouts::getHTTPTimeouts( + context->getSettingsRef(), + {context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}), + credentials); std::string columns_info; readStringBinary(columns_info, buf); From 168fbc9d7bf1767f5b03363f8efb9a70d3de6601 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 8 Feb 2023 02:17:23 +0100 Subject: [PATCH 06/14] Add a test --- tests/performance/datetime64_conversion.xml | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 tests/performance/datetime64_conversion.xml diff --git a/tests/performance/datetime64_conversion.xml b/tests/performance/datetime64_conversion.xml new file mode 100644 index 00000000000..ec13fe64242 --- /dev/null +++ b/tests/performance/datetime64_conversion.xml @@ -0,0 +1,4 @@ + + SELECT count() FROM numbers(20000000) WHERE NOT ignore(toDateTime64(rand(), 0)) + SELECT count() FROM numbers(20000000) WHERE NOT ignore(toDateTime64(rand(), 3)) + From 82a36f50c98f2c0a87d546efedef519bf36fcddc Mon Sep 17 00:00:00 2001 From: Ryadh DAHIMENE Date: Wed, 8 Feb 2023 15:11:18 +0100 Subject: [PATCH 07/14] Adding missing integrations to the list --- docs/en/engines/table-engines/integrations/index.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/engines/table-engines/integrations/index.md b/docs/en/engines/table-engines/integrations/index.md index 09e89209ea9..7a8b537aea8 100644 --- a/docs/en/engines/table-engines/integrations/index.md +++ b/docs/en/engines/table-engines/integrations/index.md @@ -22,3 +22,8 @@ List of supported integrations: - [PostgreSQL](../../../engines/table-engines/integrations/postgresql.md) - [SQLite](../../../engines/table-engines/integrations/sqlite.md) - [Hive](../../../engines/table-engines/integrations/hive.md) +- [ExternalDistributed](../../../engines/table-engines/integrations/ExternalDistributed.md) +- [MaterializedPostgreSQL](../../../engines/table-engines/integrations/materialized-postgresql.md) +- [NATS](../../../engines/table-engines/integrations/nats.md) +- [DeltaLake](../../../engines/table-engines/integrations/deltalake.md) +- [Hudi](../../../engines/table-engines/integrations/hudi.md) From cb45d3aec1c7d761629d091360e1674dd285e48e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Feb 2023 19:00:17 +0300 Subject: [PATCH 08/14] Update Dockerfile.ubuntu --- docker/server/Dockerfile.ubuntu | 1 - 1 file changed, 1 deletion(-) diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index beb70cbe662..ba2d7430e06 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -16,7 +16,6 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list gnupg \ locales \ wget \ - yasm \ tzdata \ && apt-get clean From 9314c90b0531a49daa4912d4c2137a5cc6fb3535 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 8 Feb 2023 12:28:02 +0100 Subject: [PATCH 09/14] add 00002_log_and_exception_messages_formatting back --- ...nd_exception_messages_formatting.reference | 15 +++++ ..._log_and_exception_messages_formatting.sql | 64 +++++++++++++++++++ 2 files changed, 79 insertions(+) create mode 100644 tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference create mode 100644 tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference new file mode 100644 index 00000000000..032d7e396ff --- /dev/null +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.reference @@ -0,0 +1,15 @@ +runtime messages 0.001 +runtime exceptions 0.05 +messages shorter than 10 10 +messages shorter than 16 40 +exceptions shorter than 30 125 +noisy messages 0.3 +noisy Trace messages 0.16 +noisy Debug messages 0.09 +noisy Info messages 0.05 +noisy Warning messages 0.01 +noisy Error messages 0.02 +no Fatal messages 0 +number of too noisy messages 3 +number of noisy messages 10 +incorrect patterns 15 diff --git a/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql new file mode 100644 index 00000000000..bb2a8ce8b98 --- /dev/null +++ b/tests/queries/0_stateless/00002_log_and_exception_messages_formatting.sql @@ -0,0 +1,64 @@ +-- Tags: no-parallel, no-fasttest +-- no-parallel because we want to run this test when most of the other tests already passed + +-- If this test fails, see the "Top patterns of log messages" diagnostics in the end of run.log + +system flush logs; +drop table if exists logs; +create view logs as select * from system.text_log where now() - toIntervalMinute(120) < event_time; + +-- Check that we don't have too many messages formatted with fmt::runtime or strings concatenation. +-- 0.001 threshold should be always enough, the value was about 0.00025 +select 'runtime messages', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.001) from logs; + +-- Check the same for exceptions. The value was 0.03 +select 'runtime exceptions', max2(coalesce(sum(length(message_format_string) = 0) / countOrNull(), 0), 0.05) from logs where message like '%DB::Exception%'; + +-- Check that we don't have too many short meaningless message patterns. +select 'messages shorter than 10', max2(countDistinctOrDefault(message_format_string), 10) from logs where length(message_format_string) < 10; + +-- Same as above. Feel free to update the threshold or remove this query if really necessary +select 'messages shorter than 16', max2(countDistinctOrDefault(message_format_string), 40) from logs where length(message_format_string) < 16; + +-- Same as above, but exceptions must be more informative. Feel free to update the threshold or remove this query if really necessary +select 'exceptions shorter than 30', max2(countDistinctOrDefault(message_format_string), 125) from logs where length(message_format_string) < 30 and message ilike '%DB::Exception%'; + + +-- Avoid too noisy messages: top 1 message frequency must be less than 30%. We should reduce the threshold +select 'noisy messages', max2((select count() from logs group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.30); + +-- Same as above, but excluding Test level (actually finds top 1 Trace message) +with ('Access granted: {}{}', '{} -> {}') as frequent_in_tests +select 'noisy Trace messages', max2((select count() from logs where level!='Test' and message_format_string not in frequent_in_tests + group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.16); + +-- Same as above for Debug +select 'noisy Debug messages', max2((select count() from logs where level <= 'Debug' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.09); + +-- Same as above for Info +select 'noisy Info messages', max2((select count() from logs where level <= 'Information' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.05); + +-- Same as above for Warning +with ('Not enabled four letter command {}') as frequent_in_tests +select 'noisy Warning messages', max2((select countOrDefault() from logs where level = 'Warning' and message_format_string not in frequent_in_tests + group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.01); + +-- Same as above for Error +select 'noisy Error messages', max2((select countOrDefault() from logs where level = 'Error' group by message_format_string order by count() desc limit 1) / (select count() from logs), 0.02); + +select 'no Fatal messages', count() from logs where level = 'Fatal'; + + +-- Avoid too noisy messages: limit the number of messages with high frequency +select 'number of too noisy messages', max2(count(), 3) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.10); +select 'number of noisy messages', max2(count(), 10) from (select count() / (select count() from logs) as freq, message_format_string from logs group by message_format_string having freq > 0.05); + +-- Each message matches its pattern (returns 0 rows) +-- FIXME maybe we should make it stricter ('Code:%Exception: '||s||'%'), but it's not easy because of addMessage +select 'incorrect patterns', max2(countDistinct(message_format_string), 15) from ( + select message_format_string, any(message) as any_message from logs + where message not like (replaceRegexpAll(message_format_string, '{[:.0-9dfx]*}', '%') as s) + and message not like ('%Exception: '||s||'%') group by message_format_string +) where any_message not like '%Poco::Exception%'; + +drop table logs; From e2be729aea0dc4eb558a2df0a34061d5867ad007 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 9 Feb 2023 20:27:24 +0300 Subject: [PATCH 10/14] Update test.py --- tests/integration/test_replicated_merge_tree_s3_restore/test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/test_replicated_merge_tree_s3_restore/test.py b/tests/integration/test_replicated_merge_tree_s3_restore/test.py index 2181f260f32..20a7668982b 100644 --- a/tests/integration/test_replicated_merge_tree_s3_restore/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_restore/test.py @@ -251,7 +251,7 @@ def test_restore_another_bucket_path(cluster, db_atomic, zero_copy): node_another_bucket = cluster.instances["node_another_bucket"] create_restore_file(node_another_bucket, bucket="root") - node_another_bucket.restart_clickhouse() + node_another_bucket.restart_clickhouse(stop_start_wait_sec=120) create_table( node_another_bucket, "test", schema, attach=True, db_atomic=db_atomic, uuid=uuid ) From f7c7c9be319eb5fd6473e302b3527e594eeb627c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 9 Feb 2023 20:37:47 +0300 Subject: [PATCH 11/14] Update test.py --- tests/integration/test_distributed_ddl_parallel/test.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_distributed_ddl_parallel/test.py b/tests/integration/test_distributed_ddl_parallel/test.py index a3fe00623ca..6ebfe472e09 100644 --- a/tests/integration/test_distributed_ddl_parallel/test.py +++ b/tests/integration/test_distributed_ddl_parallel/test.py @@ -167,7 +167,7 @@ def test_smoke(): def test_smoke_parallel(): threads = [] - for _ in range(100): + for _ in range(50): threads.append(SafeThread(target=execute_smoke_query)) for thread in threads: thread.start() @@ -178,7 +178,7 @@ def test_smoke_parallel(): def test_smoke_parallel_dict_reload(): threads = [] - for _ in range(100): + for _ in range(90): threads.append(SafeThread(target=execute_reload_dictionary_slow_dict_3)) for thread in threads: thread.start() From b33486d7159553efa0f36b05af8063b99f6b8c72 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Thu, 9 Feb 2023 12:52:01 -0500 Subject: [PATCH 12/14] Update formats.md closes --- docs/en/interfaces/formats.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 6bbf7fb4696..c2a20ee5d9d 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -683,6 +683,11 @@ Example: ## JSONColumns {#jsoncolumns} +:::tip +The output of the JSONColumns* formats provides the ClickHouse field name and then the content of each row of the table for that field; +visually, the data is rotated 90 degrees to the left. +::: + In this format, all data is represented as a single JSON Object. Note that JSONColumns output format buffers all data in memory to output it as a single block and it can lead to high memory consumption. From 4e7fbad52e10173835e6ba3a3f364e65264bebb9 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 9 Feb 2023 21:00:14 +0300 Subject: [PATCH 13/14] Update 00564_versioned_collapsing_merge_tree.sql --- .../0_stateless/00564_versioned_collapsing_merge_tree.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql b/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql index fdee9390642..6e2db3e6996 100644 --- a/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql +++ b/tests/queries/0_stateless/00564_versioned_collapsing_merge_tree.sql @@ -1,4 +1,4 @@ --- Tags: no-parallel +-- Tags: no-parallel, no-random-merge-tree-settings set allow_deprecated_syntax_for_merge_tree=1; set optimize_on_insert = 0; From 59d380c4a2a1f0417a9b3fceb0e725728cf45027 Mon Sep 17 00:00:00 2001 From: Larry Luo Date: Thu, 9 Feb 2023 12:29:42 -0800 Subject: [PATCH 14/14] Avoid CVE-2022-40303 CVE-2022-40304 --- contrib/libxml2 | 2 +- contrib/libxml2-cmake/CMakeLists.txt | 2 - .../linux_x86_64/include/libxml/xmlversion.h | 66 ++++++++++++------- 3 files changed, 45 insertions(+), 25 deletions(-) diff --git a/contrib/libxml2 b/contrib/libxml2 index 7846b0a677f..f507d167f17 160000 --- a/contrib/libxml2 +++ b/contrib/libxml2 @@ -1 +1 @@ -Subproject commit 7846b0a677f8d3ce72486125fa281e92ac9970e8 +Subproject commit f507d167f1755b7eaea09fb1a44d29aab828b6d1 diff --git a/contrib/libxml2-cmake/CMakeLists.txt b/contrib/libxml2-cmake/CMakeLists.txt index a84936f8e3a..b7b3f7037ec 100644 --- a/contrib/libxml2-cmake/CMakeLists.txt +++ b/contrib/libxml2-cmake/CMakeLists.txt @@ -24,7 +24,6 @@ set(SRCS "${LIBXML2_SOURCE_DIR}/xinclude.c" "${LIBXML2_SOURCE_DIR}/nanohttp.c" "${LIBXML2_SOURCE_DIR}/nanoftp.c" - "${LIBXML2_SOURCE_DIR}/DOCBparser.c" "${LIBXML2_SOURCE_DIR}/catalog.c" "${LIBXML2_SOURCE_DIR}/globals.c" "${LIBXML2_SOURCE_DIR}/threads.c" @@ -36,7 +35,6 @@ set(SRCS "${LIBXML2_SOURCE_DIR}/xmlschemastypes.c" "${LIBXML2_SOURCE_DIR}/xmlunicode.c" "${LIBXML2_SOURCE_DIR}/triostr.c" - #"${LIBXML2_SOURCE_DIR}/trio.c" "${LIBXML2_SOURCE_DIR}/xmlreader.c" "${LIBXML2_SOURCE_DIR}/relaxng.c" "${LIBXML2_SOURCE_DIR}/dict.c" diff --git a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h index 52f62214324..9eabfaa50c8 100644 --- a/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h +++ b/contrib/libxml2-cmake/linux_x86_64/include/libxml/xmlversion.h @@ -1,6 +1,6 @@ /* - * Summary: compile-time version informations - * Description: compile-time version informations for the XML library + * Summary: compile-time version information + * Description: compile-time version information for the XML library * * Copy: See Copyright for the status of this software. * @@ -29,28 +29,28 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * the version string like "1.2.3" */ -#define LIBXML_DOTTED_VERSION "2.9.8" +#define LIBXML_DOTTED_VERSION "2.10.3" /** * LIBXML_VERSION: * * the version number: 1.2.3 value is 10203 */ -#define LIBXML_VERSION 20908 +#define LIBXML_VERSION 21003 /** * LIBXML_VERSION_STRING: * * the version number string, 1.2.3 value is "10203" */ -#define LIBXML_VERSION_STRING "20908" +#define LIBXML_VERSION_STRING "21003" /** * LIBXML_VERSION_EXTRA: * - * extra version information, used to show a CVS compilation + * extra version information, used to show a git commit description */ -#define LIBXML_VERSION_EXTRA "-GITv2.9.9-rc2-1-g6fc04d71" +#define LIBXML_VERSION_EXTRA "" /** * LIBXML_TEST_VERSION: @@ -58,7 +58,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * Macro to check that the libxml version in use is compatible with * the version the software has been compiled against */ -#define LIBXML_TEST_VERSION xmlCheckVersion(20908); +#define LIBXML_TEST_VERSION xmlCheckVersion(21003); #ifndef VMS #if 0 @@ -90,7 +90,9 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether the thread support is configured in */ -#define LIBXML_THREAD_ENABLED 1 +#if 1 +#define LIBXML_THREAD_ENABLED +#endif /** * LIBXML_THREAD_ALLOC_ENABLED: @@ -169,7 +171,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether the FTP support is configured in */ -#if 1 +#if 0 #define LIBXML_FTP_ENABLED #endif @@ -205,7 +207,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether the deprecated APIs are compiled in for compatibility */ -#if 1 +#if 0 #define LIBXML_LEGACY_ENABLED #endif @@ -227,15 +229,6 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); #define LIBXML_CATALOG_ENABLED #endif -/** - * LIBXML_DOCB_ENABLED: - * - * Whether the SGML Docbook support is configured in - */ -#if 1 -#define LIBXML_DOCB_ENABLED -#endif - /** * LIBXML_XPATH_ENABLED: * @@ -254,6 +247,15 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); #define LIBXML_XPTR_ENABLED #endif +/** + * LIBXML_XPTR_LOCS_ENABLED: + * + * Whether support for XPointer locations is configured in + */ +#if 0 +#define LIBXML_XPTR_LOCS_ENABLED +#endif + /** * LIBXML_XINCLUDE_ENABLED: * @@ -268,7 +270,7 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * * Whether iconv support is available */ -#if 0 +#if 1 #define LIBXML_ICONV_ENABLED #endif @@ -348,8 +350,10 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * LIBXML_EXPR_ENABLED: * * Whether the formal expressions interfaces are compiled in + * + * This code is unused and disabled unconditionally for now. */ -#if 1 +#if 0 #define LIBXML_EXPR_ENABLED #endif @@ -452,6 +456,15 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); # define LIBXML_ATTR_FORMAT(fmt,args) #endif +#ifndef XML_DEPRECATED +# ifdef IN_LIBXML +# define XML_DEPRECATED +# else +/* Available since at least GCC 3.1 */ +# define XML_DEPRECATED __attribute__((deprecated)) +# endif +#endif + #else /* ! __GNUC__ */ /** * ATTRIBUTE_UNUSED: @@ -471,6 +484,15 @@ XMLPUBFUN void XMLCALL xmlCheckVersion(int version); * Macro used to indicate to GCC the parameter are printf like */ #define LIBXML_ATTR_FORMAT(fmt,args) +/** + * XML_DEPRECATED: + * + * Macro used to indicate that a function, variable, type or struct member + * is deprecated. + */ +#ifndef XML_DEPRECATED +#define XML_DEPRECATED +#endif #endif /* __GNUC__ */ #ifdef __cplusplus