Merge branch 'master' of github.com:ClickHouse/ClickHouse into full-syntax-highlight

This commit is contained in:
Alexey Milovidov 2024-04-10 01:58:49 +02:00
commit 0e06a52962
305 changed files with 4789 additions and 895 deletions

View File

@ -119,7 +119,6 @@ Checks: [
'-readability-named-parameter',
'-readability-redundant-declaration',
'-readability-simplify-boolean-expr',
'-readability-static-accessed-through-instance',
'-readability-suspicious-call-argument',
'-readability-uppercase-literal-suffix',
'-readability-use-anyofallof',

View File

@ -157,7 +157,7 @@ jobs:
################################# Stage Final #################################
#
FinishCheck:
if: ${{ !failure() && !cancelled() }}
if: ${{ !failure() && !cancelled() && github.event_name != 'merge_group' }}
needs: [Tests_1, Tests_2]
runs-on: [self-hosted, style-checker]
steps:

View File

@ -213,6 +213,19 @@ namespace Net
Poco::Timespan getKeepAliveTimeout() const;
/// Returns the connection timeout for HTTP connections.
void setKeepAliveMaxRequests(int max_requests);
int getKeepAliveMaxRequests() const;
int getKeepAliveRequest() const;
bool isKeepAliveExpired(double reliability = 1.0) const;
/// Returns if the connection is expired with some margin as fraction of timeout as reliability
double getKeepAliveReliability() const;
/// Returns the current fraction of keep alive timeout when connection is considered safe to use
/// It helps to avoid situation when a client uses nearly expired connection and receives NoMessageException
virtual std::ostream & sendRequest(HTTPRequest & request);
/// Sends the header for the given HTTP request to
/// the server.
@ -345,6 +358,8 @@ namespace Net
void assign(HTTPClientSession & session);
void setKeepAliveRequest(int request);
HTTPSessionFactory _proxySessionFactory;
/// Factory to create HTTPClientSession to proxy.
private:
@ -353,6 +368,8 @@ namespace Net
Poco::UInt16 _port;
ProxyConfig _proxyConfig;
Poco::Timespan _keepAliveTimeout;
int _keepAliveCurrentRequest = 0;
int _keepAliveMaxRequests = 1000;
Poco::Timestamp _lastRequest;
bool _reconnect;
bool _mustReconnect;
@ -361,6 +378,7 @@ namespace Net
Poco::SharedPtr<std::ostream> _pRequestStream;
Poco::SharedPtr<std::istream> _pResponseStream;
static const double _defaultKeepAliveReliabilityLevel;
static ProxyConfig _globalProxyConfig;
HTTPClientSession(const HTTPClientSession &);
@ -450,9 +468,19 @@ namespace Net
return _lastRequest;
}
inline void HTTPClientSession::setLastRequest(Poco::Timestamp time)
inline double HTTPClientSession::getKeepAliveReliability() const
{
_lastRequest = time;
return _defaultKeepAliveReliabilityLevel;
}
inline int HTTPClientSession::getKeepAliveMaxRequests() const
{
return _keepAliveMaxRequests;
}
inline int HTTPClientSession::getKeepAliveRequest() const
{
return _keepAliveCurrentRequest;
}
}

View File

@ -120,6 +120,10 @@ namespace Net
/// The value is set to "Keep-Alive" if keepAlive is
/// true, or to "Close" otherwise.
void setKeepAliveTimeout(int timeout, int max_requests);
int getKeepAliveTimeout() const;
int getKeepAliveMaxRequests() const;
bool getKeepAlive() const;
/// Returns true if
/// * the message has a Connection header field and its value is "Keep-Alive"

View File

@ -44,7 +44,7 @@ namespace Net
/// - timeout: 60 seconds
/// - keepAlive: true
/// - maxKeepAliveRequests: 0
/// - keepAliveTimeout: 10 seconds
/// - keepAliveTimeout: 15 seconds
void setServerName(const std::string & serverName);
/// Sets the name and port (name:port) that the server uses to identify itself.

View File

@ -56,6 +56,8 @@ namespace Net
SocketAddress serverAddress();
/// Returns the server's address.
void setKeepAliveTimeout(Poco::Timespan keepAliveTimeout);
private:
bool _firstRequest;
Poco::Timespan _keepAliveTimeout;

View File

@ -37,6 +37,7 @@ namespace Net {
HTTPClientSession::ProxyConfig HTTPClientSession::_globalProxyConfig;
const double HTTPClientSession::_defaultKeepAliveReliabilityLevel = 0.9;
HTTPClientSession::HTTPClientSession():
@ -220,7 +221,41 @@ void HTTPClientSession::setGlobalProxyConfig(const ProxyConfig& config)
void HTTPClientSession::setKeepAliveTimeout(const Poco::Timespan& timeout)
{
_keepAliveTimeout = timeout;
if (connected())
{
throw Poco::IllegalStateException("cannot change keep alive timeout on initiated connection, "
"That value is managed privately after connection is established.");
}
_keepAliveTimeout = timeout;
}
void HTTPClientSession::setKeepAliveMaxRequests(int max_requests)
{
if (connected())
{
throw Poco::IllegalStateException("cannot change keep alive max requests on initiated connection, "
"That value is managed privately after connection is established.");
}
_keepAliveMaxRequests = max_requests;
}
void HTTPClientSession::setKeepAliveRequest(int request)
{
_keepAliveCurrentRequest = request;
}
void HTTPClientSession::setLastRequest(Poco::Timestamp time)
{
if (connected())
{
throw Poco::IllegalStateException("cannot change last request on initiated connection, "
"That value is managed privately after connection is established.");
}
_lastRequest = time;
}
@ -231,6 +266,8 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request)
clearException();
_responseReceived = false;
_keepAliveCurrentRequest += 1;
bool keepAlive = getKeepAlive();
if (((connected() && !keepAlive) || mustReconnect()) && !_host.empty())
{
@ -241,8 +278,10 @@ std::ostream& HTTPClientSession::sendRequest(HTTPRequest& request)
{
if (!connected())
reconnect();
if (!keepAlive)
request.setKeepAlive(false);
if (!request.has(HTTPMessage::CONNECTION))
request.setKeepAlive(keepAlive);
if (keepAlive && !request.has(HTTPMessage::CONNECTION_KEEP_ALIVE) && _keepAliveTimeout.totalSeconds() > 0)
request.setKeepAliveTimeout(_keepAliveTimeout.totalSeconds(), _keepAliveMaxRequests);
if (!request.has(HTTPRequest::HOST) && !_host.empty())
request.setHost(_host, _port);
if (!_proxyConfig.host.empty() && !bypassProxy())
@ -324,6 +363,17 @@ std::istream& HTTPClientSession::receiveResponse(HTTPResponse& response)
_mustReconnect = getKeepAlive() && !response.getKeepAlive();
if (!_mustReconnect)
{
/// when server sends its keep alive timeout, client has to follow that value
auto timeout = response.getKeepAliveTimeout();
if (timeout > 0)
_keepAliveTimeout = std::min(_keepAliveTimeout, Poco::Timespan(timeout, 0));
auto max_requests = response.getKeepAliveMaxRequests();
if (max_requests > 0)
_keepAliveMaxRequests = std::min(_keepAliveMaxRequests, max_requests);
}
if (!_expectResponseBody || response.getStatus() < 200 || response.getStatus() == HTTPResponse::HTTP_NO_CONTENT || response.getStatus() == HTTPResponse::HTTP_NOT_MODIFIED)
_pResponseStream = new HTTPFixedLengthInputStream(*this, 0);
else if (response.getChunkedTransferEncoding())
@ -430,15 +480,18 @@ std::string HTTPClientSession::proxyRequestPrefix() const
return result;
}
bool HTTPClientSession::isKeepAliveExpired(double reliability) const
{
Poco::Timestamp now;
return Timespan(Timestamp::TimeDiff(reliability *_keepAliveTimeout.totalMicroseconds())) <= now - _lastRequest
|| _keepAliveCurrentRequest > _keepAliveMaxRequests;
}
bool HTTPClientSession::mustReconnect() const
{
if (!_mustReconnect)
{
Poco::Timestamp now;
return _keepAliveTimeout <= now - _lastRequest;
}
else return true;
return isKeepAliveExpired(_defaultKeepAliveReliabilityLevel);
return true;
}
@ -511,14 +564,21 @@ void HTTPClientSession::assign(Poco::Net::HTTPClientSession & session)
if (buffered())
throw Poco::LogicException("assign to a session with not empty buffered data");
attachSocket(session.detachSocket());
setLastRequest(session.getLastRequest());
poco_assert(!connected());
setResolvedHost(session.getResolvedHost());
setKeepAlive(session.getKeepAlive());
setProxyConfig(session.getProxyConfig());
setTimeout(session.getConnectionTimeout(), session.getSendTimeout(), session.getReceiveTimeout());
setKeepAlive(session.getKeepAlive());
setLastRequest(session.getLastRequest());
setKeepAliveTimeout(session.getKeepAliveTimeout());
setProxyConfig(session.getProxyConfig());
_keepAliveMaxRequests = session._keepAliveMaxRequests;
_keepAliveCurrentRequest = session._keepAliveCurrentRequest;
attachSocket(session.detachSocket());
session.reset();
}

View File

@ -17,6 +17,7 @@
#include "Poco/NumberFormatter.h"
#include "Poco/NumberParser.h"
#include "Poco/String.h"
#include <format>
using Poco::NumberFormatter;
@ -179,4 +180,51 @@ bool HTTPMessage::getKeepAlive() const
}
void HTTPMessage::setKeepAliveTimeout(int timeout, int max_requests)
{
add(HTTPMessage::CONNECTION_KEEP_ALIVE, std::format("timeout={}, max={}", timeout, max_requests));
}
int parseFromHeaderValues(const std::string_view header_value, const std::string_view param_name)
{
auto param_value_pos = header_value.find(param_name);
if (param_value_pos == std::string::npos)
param_value_pos = header_value.size();
if (param_value_pos != header_value.size())
param_value_pos += param_name.size();
auto param_value_end = header_value.find(',', param_value_pos);
if (param_value_end == std::string::npos)
param_value_end = header_value.size();
auto timeout_value_substr = header_value.substr(param_value_pos, param_value_end - param_value_pos);
if (timeout_value_substr.empty())
return -1;
int value = 0;
auto [ptr, ec] = std::from_chars(timeout_value_substr.begin(), timeout_value_substr.end(), value);
if (ec == std::errc())
return value;
return -1;
}
int HTTPMessage::getKeepAliveTimeout() const
{
const std::string& ka_header = get(HTTPMessage::CONNECTION_KEEP_ALIVE, HTTPMessage::EMPTY);
static const std::string_view timeout_param = "timeout=";
return parseFromHeaderValues(ka_header, timeout_param);
}
int HTTPMessage::getKeepAliveMaxRequests() const
{
const std::string& ka_header = get(HTTPMessage::CONNECTION_KEEP_ALIVE, HTTPMessage::EMPTY);
static const std::string_view timeout_param = "max=";
return parseFromHeaderValues(ka_header, timeout_param);
}
} } // namespace Poco::Net

View File

@ -88,7 +88,18 @@ void HTTPServerConnection::run()
pHandler->handleRequest(request, response);
session.setKeepAlive(_pParams->getKeepAlive() && response.getKeepAlive() && session.canKeepAlive());
}
/// all that fuzz is all about to make session close with less timeout than 15s (set in HTTPServerParams c-tor)
if (_pParams->getKeepAlive() && response.getKeepAlive() && session.canKeepAlive())
{
int value = response.getKeepAliveTimeout();
if (value < 0)
value = request.getKeepAliveTimeout();
if (value > 0)
session.setKeepAliveTimeout(Poco::Timespan(value, 0));
}
}
else sendErrorResponse(session, HTTPResponse::HTTP_NOT_IMPLEMENTED);
}
catch (Poco::Exception&)

View File

@ -33,6 +33,12 @@ HTTPServerSession::~HTTPServerSession()
{
}
void HTTPServerSession::setKeepAliveTimeout(Poco::Timespan keepAliveTimeout)
{
_keepAliveTimeout = keepAliveTimeout;
}
bool HTTPServerSession::hasMoreRequests()
{

View File

@ -59,12 +59,3 @@ target_link_libraries (_avrocpp PRIVATE boost::headers_only boost::iostreams)
target_compile_definitions (_avrocpp PUBLIC SNAPPY_CODEC_AVAILABLE)
target_include_directories (_avrocpp PRIVATE ${SNAPPY_INCLUDE_DIR})
target_link_libraries (_avrocpp PRIVATE ch_contrib::snappy)
# create a symlink to include headers with <avro/...>
set(AVRO_INCLUDE_DIR "${CMAKE_CURRENT_BINARY_DIR}/include")
ADD_CUSTOM_TARGET(avro_symlink_headers ALL
COMMAND ${CMAKE_COMMAND} -E make_directory "${AVRO_INCLUDE_DIR}"
COMMAND ${CMAKE_COMMAND} -E create_symlink "${AVROCPP_ROOT_DIR}/api" "${AVRO_INCLUDE_DIR}/avro"
)
add_dependencies(_avrocpp avro_symlink_headers)
target_include_directories(_avrocpp SYSTEM BEFORE PUBLIC "${AVRO_INCLUDE_DIR}")

View File

@ -1,26 +1,18 @@
option (ENABLE_SSH "Enable support for SSH keys and protocol" ${ENABLE_LIBRARIES})
option (ENABLE_SSH "Enable support for libssh" ${ENABLE_LIBRARIES})
if (NOT ENABLE_SSH)
message(STATUS "Not using SSH")
message(STATUS "Not using libssh")
return()
endif()
# CMake variables needed by libssh_version.h.cmake, update them when you update libssh
set(libssh_VERSION_MAJOR 0)
set(libssh_VERSION_MINOR 9)
set(libssh_VERSION_PATCH 8)
set(LIB_SOURCE_DIR "${ClickHouse_SOURCE_DIR}/contrib/libssh")
set(LIB_BINARY_DIR "${ClickHouse_BINARY_DIR}/contrib/libssh")
# Set CMake variables which are used in libssh_version.h.cmake
project(libssh VERSION 0.9.8 LANGUAGES C)
set(LIBRARY_VERSION "4.8.8")
set(LIBRARY_SOVERSION "4")
set(CMAKE_THREAD_PREFER_PTHREADS ON)
set(THREADS_PREFER_PTHREAD_FLAG ON)
set(WITH_ZLIB OFF)
set(WITH_SYMBOL_VERSIONING OFF)
set(WITH_SERVER ON)
set(libssh_SRCS
${LIB_SOURCE_DIR}/src/agent.c
${LIB_SOURCE_DIR}/src/auth.c
@ -28,15 +20,21 @@ set(libssh_SRCS
${LIB_SOURCE_DIR}/src/bignum.c
${LIB_SOURCE_DIR}/src/buffer.c
${LIB_SOURCE_DIR}/src/callbacks.c
${LIB_SOURCE_DIR}/src/chachapoly.c
${LIB_SOURCE_DIR}/src/channels.c
${LIB_SOURCE_DIR}/src/client.c
${LIB_SOURCE_DIR}/src/config.c
${LIB_SOURCE_DIR}/src/config_parser.c
${LIB_SOURCE_DIR}/src/connect.c
${LIB_SOURCE_DIR}/src/connector.c
${LIB_SOURCE_DIR}/src/curve25519.c
${LIB_SOURCE_DIR}/src/dh.c
${LIB_SOURCE_DIR}/src/ecdh.c
${LIB_SOURCE_DIR}/src/error.c
${LIB_SOURCE_DIR}/src/external/bcrypt_pbkdf.c
${LIB_SOURCE_DIR}/src/external/blowfish.c
${LIB_SOURCE_DIR}/src/external/chacha.c
${LIB_SOURCE_DIR}/src/external/poly1305.c
${LIB_SOURCE_DIR}/src/getpass.c
${LIB_SOURCE_DIR}/src/init.c
${LIB_SOURCE_DIR}/src/kdf.c
@ -55,37 +53,32 @@ set(libssh_SRCS
${LIB_SOURCE_DIR}/src/pcap.c
${LIB_SOURCE_DIR}/src/pki.c
${LIB_SOURCE_DIR}/src/pki_container_openssh.c
${LIB_SOURCE_DIR}/src/pki_ed25519_common.c
${LIB_SOURCE_DIR}/src/poll.c
${LIB_SOURCE_DIR}/src/session.c
${LIB_SOURCE_DIR}/src/scp.c
${LIB_SOURCE_DIR}/src/session.c
${LIB_SOURCE_DIR}/src/socket.c
${LIB_SOURCE_DIR}/src/string.c
${LIB_SOURCE_DIR}/src/threads.c
${LIB_SOURCE_DIR}/src/wrapper.c
${LIB_SOURCE_DIR}/src/external/bcrypt_pbkdf.c
${LIB_SOURCE_DIR}/src/external/blowfish.c
${LIB_SOURCE_DIR}/src/external/chacha.c
${LIB_SOURCE_DIR}/src/external/poly1305.c
${LIB_SOURCE_DIR}/src/chachapoly.c
${LIB_SOURCE_DIR}/src/config_parser.c
${LIB_SOURCE_DIR}/src/token.c
${LIB_SOURCE_DIR}/src/pki_ed25519_common.c
${LIB_SOURCE_DIR}/src/wrapper.c
# some files of libssh/src/ are missing - why?
${LIB_SOURCE_DIR}/src/threads/noop.c
${LIB_SOURCE_DIR}/src/threads/pthread.c
# files missing - why?
# LIBCRYPT specific
${libssh_SRCS}
${LIB_SOURCE_DIR}/src/threads/libcrypto.c
${LIB_SOURCE_DIR}/src/pki_crypto.c
${LIB_SOURCE_DIR}/src/dh_crypto.c
${LIB_SOURCE_DIR}/src/ecdh_crypto.c
${LIB_SOURCE_DIR}/src/libcrypto.c
${LIB_SOURCE_DIR}/src/dh_crypto.c
${LIB_SOURCE_DIR}/src/pki_crypto.c
${LIB_SOURCE_DIR}/src/threads/libcrypto.c
${LIB_SOURCE_DIR}/src/options.c
${LIB_SOURCE_DIR}/src/server.c
${LIB_SOURCE_DIR}/src/bind.c
${LIB_SOURCE_DIR}/src/bind_config.c
${LIB_SOURCE_DIR}/src/options.c
${LIB_SOURCE_DIR}/src/server.c
)
if (NOT (ENABLE_OPENSSL OR ENABLE_OPENSSL_DYNAMIC))
@ -94,7 +87,7 @@ endif()
configure_file(${LIB_SOURCE_DIR}/include/libssh/libssh_version.h.cmake ${LIB_BINARY_DIR}/include/libssh/libssh_version.h @ONLY)
add_library(_ssh STATIC ${libssh_SRCS})
add_library(_ssh ${libssh_SRCS})
add_library(ch_contrib::ssh ALIAS _ssh)
target_link_libraries(_ssh PRIVATE OpenSSL::Crypto)

View File

@ -43,6 +43,8 @@ source /utils.lib
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
echo "Azure is disabled"
elif [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
echo "Azure is disabled"
else
azurite-blob --blobHost 0.0.0.0 --blobPort 10000 --debug /azurite_log &
fi
@ -139,6 +141,32 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
sudo cat /etc/clickhouse-server1/config.d/filesystem_caches_path.xml \
| sed "s|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches/</filesystem_caches_path>|<filesystem_caches_path>/var/lib/clickhouse/filesystem_caches_1/</filesystem_caches_path>|" \
> /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp
mv /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp /etc/clickhouse-server1/config.d/filesystem_caches_path.xml
sudo cat /etc/clickhouse-server1/config.d/filesystem_caches_path.xml \
| sed "s|<custom_cached_disks_base_directory replace=\"replace\">/var/lib/clickhouse/filesystem_caches/</custom_cached_disks_base_directory>|<custom_cached_disks_base_directory replace=\"replace\">/var/lib/clickhouse/filesystem_caches_1/</custom_cached_disks_base_directory>|" \
> /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp
mv /etc/clickhouse-server1/config.d/filesystem_caches_path.xml.tmp /etc/clickhouse-server1/config.d/filesystem_caches_path.xml
mkdir -p /var/run/clickhouse-server1
sudo chown clickhouse:clickhouse /var/run/clickhouse-server1
sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \
--pid-file /var/run/clickhouse-server1/clickhouse-server.pid \
-- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \
--logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \
--tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \
--mysql_port 19004 --postgresql_port 19005 \
--keeper_server.tcp_port 19181 --keeper_server.server_id 2 \
--prometheus.port 19988 \
--macros.replica r2 # It doesn't work :(
MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours)
MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited)
fi
# Wait for the server to start, but not for too long.
for _ in {1..100}
@ -185,6 +213,10 @@ function run_tests()
ADDITIONAL_OPTIONS+=('--s3-storage')
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--shared-catalog')
fi
if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then
ADDITIONAL_OPTIONS+=('--replicated-database')
# Too many tests fail for DatabaseReplicated in parallel.
@ -266,6 +298,12 @@ do
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
err=$( { clickhouse-client --port 19000 -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst; } 2>&1 )
echo "$err"
[[ "0" != "${#err}" ]] && failed_to_save_logs=1
fi
done
# Stop server so we can safely read data with clickhouse-local.
@ -277,6 +315,10 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
sudo clickhouse stop --pid-path /var/run/clickhouse-server2 ||:
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
sudo clickhouse stop --pid-path /var/run/clickhouse-server1 ||:
fi
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server.log ||:
rg -A50 -Fa "============" /var/log/clickhouse-server/stderr.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server.log > /test_output/clickhouse-server.log.zst &
@ -304,6 +346,10 @@ if [ $failed_to_save_logs -ne 0 ]; then
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
clickhouse-local --path /var/lib/clickhouse2/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.2.tsv.zst ||:
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
clickhouse-local --path /var/lib/clickhouse1/ --only-system-tables --stacktrace -q "select * from system.$table format TSVWithNamesAndTypes" | zstd --threads=0 > /test_output/$table.1.tsv.zst ||:
fi
done
fi
@ -343,3 +389,10 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]
tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||:
tar -chf /test_output/coordination2.tar /var/lib/clickhouse2/coordination ||:
fi
if [[ -n "$USE_SHARED_CATALOG" ]] && [[ "$USE_SHARED_CATALOG" -eq 1 ]]; then
rg -Fa "<Fatal>" /var/log/clickhouse-server/clickhouse-server1.log ||:
zstd --threads=0 < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.zst ||:
mv /var/log/clickhouse-server/stderr1.log /test_output/ ||:
tar -chf /test_output/coordination1.tar /var/lib/clickhouse1/coordination ||:
fi

View File

@ -68,6 +68,12 @@ In the results of `SELECT` query, the values of `AggregateFunction` type have im
## Example of an Aggregated Materialized View {#example-of-an-aggregated-materialized-view}
The following examples assumes that you have a database named `test` so make sure you create that if it doesn't already exist:
```sql
CREATE DATABASE test;
```
We will create the table `test.visits` that contain the raw data:
``` sql
@ -80,17 +86,24 @@ CREATE TABLE test.visits
) ENGINE = MergeTree ORDER BY (StartDate, CounterID);
```
Next, we need to create an `AggregatingMergeTree` table that will store `AggregationFunction`s that keep track of the total number of visits and the number of unique users.
`AggregatingMergeTree` materialized view that watches the `test.visits` table, and use the `AggregateFunction` type:
``` sql
CREATE MATERIALIZED VIEW test.mv_visits
(
CREATE TABLE test.agg_visits (
StartDate DateTime64 NOT NULL,
CounterID UInt64,
Visits AggregateFunction(sum, Nullable(Int32)),
Users AggregateFunction(uniq, Nullable(Int32))
)
ENGINE = AggregatingMergeTree() ORDER BY (StartDate, CounterID)
ENGINE = AggregatingMergeTree() ORDER BY (StartDate, CounterID);
```
And then let's create a materialized view that populates `test.agg_visits` from `test.visits` :
```sql
CREATE MATERIALIZED VIEW test.visits_mv TO test.agg_visits
AS SELECT
StartDate,
CounterID,
@ -104,25 +117,45 @@ Inserting data into the `test.visits` table.
``` sql
INSERT INTO test.visits (StartDate, CounterID, Sign, UserID)
VALUES (1667446031, 1, 3, 4)
INSERT INTO test.visits (StartDate, CounterID, Sign, UserID)
VALUES (1667446031, 1, 6, 3)
VALUES (1667446031000, 1, 3, 4), (1667446031000, 1, 6, 3);
```
The data is inserted in both the table and the materialized view `test.mv_visits`.
The data is inserted in both `test.visits` and `test.agg_visits`.
To get the aggregated data, we need to execute a query such as `SELECT ... GROUP BY ...` from the materialized view `test.mv_visits`:
``` sql
```sql
SELECT
StartDate,
sumMerge(Visits) AS Visits,
uniqMerge(Users) AS Users
FROM test.mv_visits
FROM test.agg_visits
GROUP BY StartDate
ORDER BY StartDate;
```
```text
┌───────────────StartDate─┬─Visits─┬─Users─┐
│ 2022-11-03 03:27:11.000 │ 9 │ 2 │
└─────────────────────────┴────────┴───────┘
```
And how about if we add another couple of records to `test.visits`, but this time we'll use a different timestamp for one of the records:
```sql
INSERT INTO test.visits (StartDate, CounterID, Sign, UserID)
VALUES (1669446031000, 2, 5, 10), (1667446031000, 3, 7, 5);
```
If we then run the `SELECT` query again, we'll see the following output:
```text
┌───────────────StartDate─┬─Visits─┬─Users─┐
│ 2022-11-03 03:27:11.000 │ 16 │ 3 │
│ 2022-11-26 07:00:31.000 │ 5 │ 1 │
└─────────────────────────┴────────┴───────┘
```
## Related Content
- Blog: [Using Aggregate Combinators in ClickHouse](https://clickhouse.com/blog/aggregate-functions-combinators-in-clickhouse-for-arrays-maps-and-states)

View File

@ -15,9 +15,9 @@ The `uniqCombined` function is a good choice for calculating the number of diffe
**Arguments**
The function takes a variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.
- `HLL_precision`: The base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optional, you can use the function as `uniqCombined(x[, ...])`. The default value for `HLL_precision` is 17, which is effectively 96 KiB of space (2^17 cells, 6 bits each).
- `X`: A variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.
`HLL_precision` is the base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optional, you can use the function as `uniqCombined(x[, ...])`. The default value for `HLL_precision` is 17, which is effectively 96 KiB of space (2^17 cells, 6 bits each).
**Returned value**
@ -25,26 +25,43 @@ The function takes a variable number of parameters. Parameters can be `Tuple`, `
**Implementation details**
Function:
The `uniqCombined` function:
- Calculates a hash (64-bit hash for `String` and 32-bit otherwise) for all parameters in the aggregate, then uses it in calculations.
- Uses a combination of three algorithms: array, hash table, and HyperLogLog with an error correction table.
For a small number of distinct elements, an array is used. When the set size is larger, a hash table is used. For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory.
- For a small number of distinct elements, an array is used.
- When the set size is larger, a hash table is used.
- For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory.
- Provides the result deterministically (it does not depend on the query processing order).
:::note
Since it uses 32-bit hash for non-`String` type, the result will have very high error for cardinalities significantly larger than `UINT_MAX` (error will raise quickly after a few tens of billions of distinct values), hence in this case you should use [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64)
Since it uses a 32-bit hash for non-`String` types, the result will have very high error for cardinalities significantly larger than `UINT_MAX` (error will raise quickly after a few tens of billions of distinct values), hence in this case you should use [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64).
:::
Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined`:
Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined` function:
- Consumes several times less memory.
- Calculates with several times higher accuracy.
- Usually has slightly lower performance. In some scenarios, `uniqCombined` can perform better than `uniq`, for example, with distributed queries that transmit a large number of aggregation states over the network.
**Example**
Query:
```sql
SELECT uniqCombined(number) FROM numbers(1e6);
```
Result:
```response
┌─uniqCombined(number)─┐
│ 1001148 │ -- 1.00 million
└──────────────────────┘
```
See the example section of [uniqCombined64](../../../sql-reference/aggregate-functions/reference/uniqcombined64.md#agg_function-uniqcombined64) for an example of the difference between `uniqCombined` and `uniqCombined64` for much larger inputs.
**See Also**
- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq)

View File

@ -5,4 +5,78 @@ sidebar_position: 193
# uniqCombined64
Same as [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined), but uses 64-bit hash for all data types.
Calculates the approximate number of different argument values. It is the same as [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md#agg_function-uniqcombined), but uses a 64-bit hash for all data types rather than just for the String data type.
``` sql
uniqCombined64(HLL_precision)(x[, ...])
```
**Parameters**
- `HLL_precision`: The base-2 logarithm of the number of cells in [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog). Optionally, you can use the function as `uniqCombined64(x[, ...])`. The default value for `HLL_precision` is 17, which is effectively 96 KiB of space (2^17 cells, 6 bits each).
- `X`: A variable number of parameters. Parameters can be `Tuple`, `Array`, `Date`, `DateTime`, `String`, or numeric types.
**Returned value**
- A number [UInt64](../../../sql-reference/data-types/int-uint.md)-type number.
**Implementation details**
The `uniqCombined64` function:
- Calculates a hash (64-bit hash for all data types) for all parameters in the aggregate, then uses it in calculations.
- Uses a combination of three algorithms: array, hash table, and HyperLogLog with an error correction table.
- For a small number of distinct elements, an array is used.
- When the set size is larger, a hash table is used.
- For a larger number of elements, HyperLogLog is used, which will occupy a fixed amount of memory.
- Provides the result deterministically (it does not depend on the query processing order).
:::note
Since it uses 64-bit hash for all types, the result does not suffer from very high error for cardinalities significantly larger than `UINT_MAX` like [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md) does, which uses a 32-bit hash for non-`String` types.
:::
Compared to the [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq) function, the `uniqCombined64` function:
- Consumes several times less memory.
- Calculates with several times higher accuracy.
**Example**
In the example below `uniqCombined64` is run on `1e10` different numbers returning a very close approximation of the number of different argument values.
Query:
```sql
SELECT uniqCombined64(number) FROM numbers(1e10);
```
Result:
```response
┌─uniqCombined64(number)─┐
│ 9998568925 │ -- 10.00 billion
└────────────────────────┘
```
By comparison the `uniqCombined` function returns a rather poor approximation for an input this size.
Query:
```sql
SELECT uniqCombined(number) FROM numbers(1e10);
```
Result:
```response
┌─uniqCombined(number)─┐
│ 5545308725 │ -- 5.55 billion
└──────────────────────┘
```
**See Also**
- [uniq](../../../sql-reference/aggregate-functions/reference/uniq.md#agg_function-uniq)
- [uniqCombined](../../../sql-reference/aggregate-functions/reference/uniqcombined.md)
- [uniqHLL12](../../../sql-reference/aggregate-functions/reference/uniqhll12.md#agg_function-uniqhll12)
- [uniqExact](../../../sql-reference/aggregate-functions/reference/uniqexact.md#agg_function-uniqexact)
- [uniqTheta](../../../sql-reference/aggregate-functions/reference/uniqthetasketch.md#agg_function-uniqthetasketch)

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/aggregatefunction
sidebar_position: 53
sidebar_position: 46
sidebar_label: AggregateFunction
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/array
sidebar_position: 52
sidebar_position: 32
sidebar_label: Array(T)
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/boolean
sidebar_position: 43
sidebar_position: 22
sidebar_label: Boolean
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/date
sidebar_position: 47
sidebar_position: 12
sidebar_label: Date
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/date32
sidebar_position: 48
sidebar_position: 14
sidebar_label: Date32
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/datetime
sidebar_position: 48
sidebar_position: 16
sidebar_label: DateTime
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/datetime64
sidebar_position: 49
sidebar_position: 18
sidebar_label: DateTime64
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/decimal
sidebar_position: 42
sidebar_position: 6
sidebar_label: Decimal
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/enum
sidebar_position: 50
sidebar_position: 20
sidebar_label: Enum
---

View File

@ -1,10 +1,10 @@
---
slug: /en/sql-reference/data-types/fixedstring
sidebar_position: 45
sidebar_position: 10
sidebar_label: FixedString(N)
---
# FixedString
# FixedString(N)
A fixed-length string of `N` bytes (neither characters nor code points).

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/float
sidebar_position: 41
sidebar_position: 4
sidebar_label: Float32, Float64
---

View File

@ -1,8 +1,8 @@
---
slug: /en/sql-reference/data-types/geo
sidebar_position: 62
sidebar_position: 54
sidebar_label: Geo
title: "Geo Data Types"
title: "Geometric"
---
ClickHouse supports data types for representing geographical objects — locations, lands, etc.

View File

@ -1,10 +1,10 @@
---
slug: /en/sql-reference/data-types/
sidebar_label: List of data types
sidebar_position: 37
sidebar_position: 1
---
# ClickHouse Data Types
# Data Types in ClickHouse
ClickHouse can store various kinds of data in table cells. This section describes the supported data types and special considerations for using and/or implementing them if any.

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/int-uint
sidebar_position: 40
sidebar_position: 2
sidebar_label: UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/ipv4
sidebar_position: 59
sidebar_position: 28
sidebar_label: IPv4
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/ipv6
sidebar_position: 60
sidebar_position: 30
sidebar_label: IPv6
---

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/json
sidebar_position: 54
sidebar_position: 26
sidebar_label: JSON
---

View File

@ -1,10 +1,10 @@
---
slug: /en/sql-reference/data-types/lowcardinality
sidebar_position: 51
sidebar_label: LowCardinality
sidebar_position: 42
sidebar_label: LowCardinality(T)
---
# LowCardinality
# LowCardinality(T)
Changes the internal representation of other data types to be dictionary-encoded.

View File

@ -1,12 +1,12 @@
---
slug: /en/sql-reference/data-types/map
sidebar_position: 65
sidebar_label: Map(key, value)
sidebar_position: 36
sidebar_label: Map(K, V)
---
# Map(key, value)
# Map(K, V)
`Map(key, value)` data type stores `key:value` pairs.
`Map(K, V)` data type stores `key:value` pairs.
**Parameters**

View File

@ -1,27 +0,0 @@
---
slug: /en/sql-reference/data-types/multiword-types
sidebar_position: 61
sidebar_label: Multiword Type Names
title: "Multiword Types"
---
When creating tables, you can use data types with a name consisting of several words. This is implemented for better SQL compatibility.
## Multiword Types Support
| Multiword types | Simple types |
|----------------------------------|--------------------------------------------------------------|
| DOUBLE PRECISION | [Float64](../../sql-reference/data-types/float.md) |
| CHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NCHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NCHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR | [String](../../sql-reference/data-types/string.md) |
| BINARY LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| BINARY VARYING | [String](../../sql-reference/data-types/string.md) |

View File

@ -1,7 +1,7 @@
---
slug: /en/sql-reference/data-types/nullable
sidebar_position: 55
sidebar_label: Nullable
sidebar_position: 44
sidebar_label: Nullable(T)
---
# Nullable(T)

View File

@ -1,5 +1,7 @@
---
slug: /en/sql-reference/data-types/simpleaggregatefunction
sidebar_position: 48
sidebar_label: SimpleAggregateFunction
---
# SimpleAggregateFunction

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/string
sidebar_position: 44
sidebar_position: 8
sidebar_label: String
---
@ -13,7 +13,7 @@ When creating tables, numeric parameters for string fields can be set (e.g. `VAR
Aliases:
- `String``LONGTEXT`, `MEDIUMTEXT`, `TINYTEXT`, `TEXT`, `LONGBLOB`, `MEDIUMBLOB`, `TINYBLOB`, `BLOB`, `VARCHAR`, `CHAR`.
- `String``LONGTEXT`, `MEDIUMTEXT`, `TINYTEXT`, `TEXT`, `LONGBLOB`, `MEDIUMBLOB`, `TINYBLOB`, `BLOB`, `VARCHAR`, `CHAR`, `CHAR LARGE OBJECT`, `CHAR VARYING`, `CHARACTER LARGE OBJECT`, `CHARACTER VARYING`, `NCHAR LARGE OBJECT`, `NCHAR VARYING`, `NATIONAL CHARACTER LARGE OBJECT`, `NATIONAL CHARACTER VARYING`, `NATIONAL CHAR VARYING`, `NATIONAL CHARACTER`, `NATIONAL CHAR`, `BINARY LARGE OBJECT`, `BINARY VARYING`,
## Encodings

View File

@ -1,10 +1,10 @@
---
slug: /en/sql-reference/data-types/tuple
sidebar_position: 54
sidebar_position: 34
sidebar_label: Tuple(T1, T2, ...)
---
# Tuple(T1, T2, )
# Tuple(T1, T2, ...)
A tuple of elements, each having an individual [type](../../sql-reference/data-types/index.md#data_types). Tuple must contain at least one element.

View File

@ -1,6 +1,6 @@
---
slug: /en/sql-reference/data-types/uuid
sidebar_position: 46
sidebar_position: 24
sidebar_label: UUID
---

View File

@ -1,10 +1,10 @@
---
slug: /en/sql-reference/data-types/variant
sidebar_position: 55
sidebar_label: Variant
sidebar_position: 40
sidebar_label: Variant(T1, T2, ...)
---
# Variant(T1, T2, T3, ...)
# Variant(T1, T2, ...)
This type represents a union of other data types. Type `Variant(T1, T2, ..., TN)` means that each row of this type
has a value of either type `T1` or `T2` or ... or `TN` or none of them (`NULL` value).

View File

@ -81,6 +81,43 @@ Result:
│ 2.23606797749979 │
└──────────────────┘
```
## L2SquaredNorm
Calculates the square root of the sum of the squares of the vector values (the [L2Norm](#l2norm)) squared.
**Syntax**
```sql
L2SquaredNorm(vector)
```
Alias: `normL2Squared`.
***Arguments**
- `vector` — [Tuple](../../sql-reference/data-types/tuple.md) or [Array](../../sql-reference/data-types/array.md).
**Returned value**
- L2-norm squared.
Type: [Float](../../sql-reference/data-types/float.md).
**Example**
Query:
```sql
SELECT L2SquaredNorm((1, 2));
```
Result:
```text
┌─L2SquaredNorm((1, 2))─┐
│ 5 │
└───────────────────────┘
```
## LinfNorm

View File

@ -594,6 +594,45 @@ Calculates JumpConsistentHash form a UInt64.
Accepts two arguments: a UInt64-type key and the number of buckets. Returns Int32.
For more information, see the link: [JumpConsistentHash](https://arxiv.org/pdf/1406.2294.pdf)
## kostikConsistentHash
An O(1) time and space consistent hash algorithm by Konstantin 'kostik' Oblakov. Previously `yandexConsistentHash`.
**Syntax**
```sql
kostikConsistentHash(input, n)
```
Alias: `yandexConsistentHash` (left for backwards compatibility sake).
**Parameters**
- `input`: A UInt64-type key [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
- `n`: Number of buckets. [UInt16](/docs/en/sql-reference/data-types/int-uint.md).
**Returned value**
- A [UInt16](/docs/en/sql-reference/data-types/int-uint.md) data type hash value.
**Implementation details**
It is efficient only if n <= 32768.
**Example**
Query:
```sql
SELECT kostikConsistentHash(16045690984833335023, 2);
```
```response
┌─kostikConsistentHash(16045690984833335023, 2)─┐
│ 1 │
└───────────────────────────────────────────────┘
```
## murmurHash2_32, murmurHash2_64
Produces a [MurmurHash2](https://github.com/aappleby/smhasher) hash value.
@ -1153,6 +1192,42 @@ Result:
└────────────┘
```
## wyHash64
Produces a 64-bit [wyHash64](https://github.com/wangyi-fudan/wyhash) hash value.
**Syntax**
```sql
wyHash64(string)
```
**Arguments**
- `string` — String. [String](/docs/en/sql-reference/data-types/string.md).
**Returned value**
- Hash value.
Type: [UInt64](/docs/en/sql-reference/data-types/int-uint.md).
**Example**
Query:
```sql
SELECT wyHash64('ClickHouse') AS Hash;
```
Result:
```response
┌─────────────────Hash─┐
│ 12336419557878201794 │
└──────────────────────┘
```
## ngramMinHash
Splits a ASCII string into n-grams of `ngramsize` symbols and calculates hash values for each n-gram. Uses `hashnum` minimum hashes to calculate the minimum hash and `hashnum` maximum hashes to calculate the maximum hash. Returns a tuple with these hashes. Is case sensitive.

View File

@ -6,14 +6,17 @@ sidebar_label: Searching in Strings
# Functions for Searching in Strings
All functions in this section search by default case-sensitively. Case-insensitive search is usually provided by separate function variants.
Note that case-insensitive search follows the lowercase-uppercase rules of the English language. E.g. Uppercased `i` in English language is
`I` whereas in Turkish language it is `İ` - results for languages other than English may be unexpected.
All functions in this section search case-sensitively by default. Case-insensitive search is usually provided by separate function variants.
Functions in this section also assume that the searched string and the search string are single-byte encoded text. If this assumption is
:::note
Case-insensitive search follows the lowercase-uppercase rules of the English language. E.g. Uppercased `i` in the English language is
`I` whereas in the Turkish language it is `İ` - results for languages other than English may be unexpected.
:::
Functions in this section also assume that the searched string (referred to in this section as `haystack`) and the search string (referred to in this section as `needle`) are single-byte encoded text. If this assumption is
violated, no exception is thrown and results are undefined. Search with UTF-8 encoded strings is usually provided by separate function
variants. Likewise, if a UTF-8 function variant is used and the input strings are not UTF-8 encoded text, no exception is thrown and the
results are undefined. Note that no automatic Unicode normalization is performed, you can use the
results are undefined. Note that no automatic Unicode normalization is performed, however you can use the
[normalizeUTF8*()](https://clickhouse.com/docs/en/sql-reference/functions/string-functions/) functions for that.
[General strings functions](string-functions.md) and [functions for replacing in strings](string-replace-functions.md) are described separately.
@ -54,6 +57,8 @@ Type: `Integer`.
**Examples**
Query:
``` sql
SELECT position('Hello, world!', '!');
```
@ -68,6 +73,8 @@ Result:
Example with `start_pos` argument:
Query:
``` sql
SELECT
position('Hello, world!', 'o', 1),
@ -84,6 +91,8 @@ Result:
Example for `needle IN haystack` syntax:
Query:
```sql
SELECT 6 = position('/' IN s) FROM (SELECT 'Hello/World' AS s);
```
@ -98,6 +107,8 @@ Result:
Examples with empty `needle` substring:
Query:
``` sql
SELECT
position('abc', ''),
@ -109,6 +120,8 @@ SELECT
position('abc', '', 5)
```
Result:
``` text
┌─position('abc', '')─┬─position('abc', '', 0)─┬─position('abc', '', 1)─┬─position('abc', '', 2)─┬─position('abc', '', 3)─┬─position('abc', '', 4)─┬─position('abc', '', 5)─┐
│ 1 │ 1 │ 1 │ 2 │ 3 │ 4 │ 0 │
@ -132,7 +145,23 @@ locate(needle, haystack[, start_pos])
## positionCaseInsensitive
Like [position](#position) but searches case-insensitively.
A case insensitive invariant of [position](#position).
**Example**
Query:
``` sql
SELECT position('Hello, world!', 'hello');
```
Result:
``` text
┌─position('Hello, world!', 'hello')─┐
│ 0 │
└────────────────────────────────────┘
```
## positionUTF8
@ -142,6 +171,8 @@ Like [position](#position) but assumes `haystack` and `needle` are UTF-8 encoded
Function `positionUTF8` correctly counts character `ö` (represented by two points) as a single Unicode codepoint:
Query:
``` sql
SELECT positionUTF8('Motörhead', 'r');
```
@ -175,14 +206,17 @@ multiSearchAllPositions(haystack, [needle1, needle2, ..., needleN])
**Arguments**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. Array
- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned values**
- Array of the starting position in bytes and counting from 1 (if the substring was found) or 0 (if the substring was not found)
- Array of the starting position in bytes and counting from 1, if the substring was found.
- 0, if the substring was not found.
**Example**
Query:
``` sql
SELECT multiSearchAllPositions('Hello, World!', ['hello', '!', 'world']);
```
@ -194,45 +228,535 @@ Result:
│ [0,13,0] │
└───────────────────────────────────────────────────────────────────┘
```
## multiSearchAllPositionsCaseInsensitive
## multiSearchAllPositionsUTF8
Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle`-s are UTF-8 encoded strings.
## multiSearchFirstPosition
Like `position` but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings.
Functions `multiSearchFirstPositionCaseInsensitive`, `multiSearchFirstPositionUTF8` and `multiSearchFirstPositionCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
Like [multiSearchAllPositions](#multisearchallpositions) but ignores case.
**Syntax**
```sql
multiSearchFirstPosition(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
multiSearchAllPositionsCaseInsensitive(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- Array of the starting position in bytes and counting from 1 (if the substring was found).
- 0 if the substring was not found.
**Example**
Query:
```sql
SELECT multiSearchAllPositionsCaseInsensitive('ClickHouse',['c','h']);
```
Result:
```response
["1","6"]
```
## multiSearchAllPositionsUTF8
Like [multiSearchAllPositions](#multiSearchAllPositions) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings.
**Syntax**
```sql
multiSearchAllPositionsUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 encoded string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- Array of the starting position in bytes and counting from 1 (if the substring was found).
- 0 if the substring was not found.
**Example**
Given `ClickHouse` as a UTF-8 string, find the positions of `C` (`\x43`) and `H` (`\x48`).
Query:
```sql
SELECT multiSearchAllPositionsUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x43','\x48']);
```
Result:
```response
["1","6"]
```
## multiSearchAllPositionsCaseInsensitiveUTF8
Like [multiSearchAllPositionsUTF8](#multisearchallpositionsutf8) but ignores case.
**Syntax**
```sql
multiSearchAllPositionsCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 encoded string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — UTF-8 encoded substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- Array of the starting position in bytes and counting from 1 (if the substring was found).
- 0 if the substring was not found.
**Example**
Given `ClickHouse` as a UTF-8 string, find the positions of `c` (`\x63`) and `h` (`\x68`).
Query:
```sql
SELECT multiSearchAllPositionsCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x63','\x68']);
```
Result:
```response
["1","6"]
```
## multiSearchFirstPosition
Like [`position`](#position) but returns the leftmost offset in a `haystack` string which matches any of multiple `needle` strings.
Functions [`multiSearchFirstPositionCaseInsensitive`](#multiSearchFirstPositionCaseInsensitive), [`multiSearchFirstPositionUTF8`](#multiSearchFirstPositionUTF8) and [`multiSearchFirstPositionCaseInsensitiveUTF8`](#multiSearchFirstPositionCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function.
**Syntax**
```sql
multiSearchFirstPosition(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings.
- 0, if there was no match.
**Example**
Query:
```sql
SELECT multiSearchFirstPosition('Hello World',['llo', 'Wor', 'ld']);
```
Result:
```response
3
```
## multiSearchFirstPositionCaseInsensitive
Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but ignores case.
**Syntax**
```sql
multiSearchFirstPositionCaseInsensitive(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Array of substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings.
- 0, if there was no match.
**Example**
Query:
```sql
SELECT multiSearchFirstPositionCaseInsensitive('HELLO WORLD',['wor', 'ld', 'ello']);
```
Result:
```response
2
```
## multiSearchFirstPositionUTF8
Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings.
**Syntax**
```sql
multiSearchFirstPositionUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings.
- 0, if there was no match.
**Example**
Find the leftmost offset in UTF-8 string `hello world` which matches any of the given needles.
Query:
```sql
SELECT multiSearchFirstPositionUTF8('\x68\x65\x6c\x6c\x6f\x20\x77\x6f\x72\x6c\x64',['wor', 'ld', 'ello']);
```
Result:
```response
2
```
## multiSearchFirstPositionCaseInsensitiveUTF8
Like [`multiSearchFirstPosition`](#multiSearchFirstPosition) but assumes `haystack` and `needle` to be UTF-8 strings and ignores case.
**Syntax**
```sql
multiSearchFirstPositionCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md)
**Returned value**
- Leftmost offset in a `haystack` string which matches any of multiple `needle` strings, ignoring case.
- 0, if there was no match.
**Example**
Find the leftmost offset in UTF-8 string `HELLO WORLD` which matches any of the given needles.
Query:
```sql
SELECT multiSearchFirstPositionCaseInsensitiveUTF8('\x48\x45\x4c\x4c\x4f\x20\x57\x4f\x52\x4c\x44',['wor', 'ld', 'ello']);
```
Result:
```response
2
```
## multiSearchFirstIndex
Returns the index `i` (starting from 1) of the leftmost found needle<sub>i</sub> in the string `haystack` and 0 otherwise.
Functions `multiSearchFirstIndexCaseInsensitive`, `multiSearchFirstIndexUTF8` and `multiSearchFirstIndexCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
Functions [`multiSearchFirstIndexCaseInsensitive`](#multiSearchFirstIndexCaseInsensitive), [`multiSearchFirstIndexUTF8`](#multiSearchFirstIndexUTF8) and [`multiSearchFirstIndexCaseInsensitiveUTF8`](#multiSearchFirstIndexCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function.
**Syntax**
```sql
multiSearchFirstIndex(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
multiSearchFirstIndex(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- index (starting from 1) of the leftmost found needle.
- 0, if there was no match.
**Example**
Query:
```sql
SELECT multiSearchFirstIndex('Hello World',['World','Hello']);
```
## multiSearchAny {#multisearchany}
Result:
```response
1
```
## multiSearchFirstIndexCaseInsensitive
Returns the index `i` (starting from 1) of the leftmost found needle<sub>i</sub> in the string `haystack` and 0 otherwise. Ignores case.
**Syntax**
```sql
multiSearchFirstIndexCaseInsensitive(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- index (starting from 1) of the leftmost found needle.
- 0, if there was no match.
**Example**
Query:
```sql
SELECT multiSearchFirstIndexCaseInsensitive('hElLo WoRlD',['World','Hello']);
```
Result:
```response
1
```
## multiSearchFirstIndexUTF8
Returns the index `i` (starting from 1) of the leftmost found needle<sub>i</sub> in the string `haystack` and 0 otherwise. Assumes `haystack` and `needle` are UTF-8 encoded strings.
**Syntax**
```sql
multiSearchFirstIndexUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md)
**Returned value**
- index (starting from 1) of the leftmost found needle.
- 0, if there was no match.
**Example**
Given `Hello World` as a UTF-8 string, find the first index of UTF-8 strings `Hello` and `World`.
Query:
```sql
SELECT multiSearchFirstIndexUTF8('\x48\x65\x6c\x6c\x6f\x20\x57\x6f\x72\x6c\x64',['\x57\x6f\x72\x6c\x64','\x48\x65\x6c\x6c\x6f']);
```
Result:
```response
1
```
## multiSearchFirstIndexCaseInsensitiveUTF8
Returns the index `i` (starting from 1) of the leftmost found needle<sub>i</sub> in the string `haystack` and 0 otherwise. Assumes `haystack` and `needle` are UTF-8 encoded strings. Ignores case.
**Syntax**
```sql
multiSearchFirstIndexCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Array of UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- index (starting from 1) of the leftmost found needle.
- 0, if there was no match.
**Example**
Given `HELLO WORLD` as a UTF-8 string, find the first index of UTF-8 strings `hello` and `world`.
Query:
```sql
SELECT multiSearchFirstIndexCaseInsensitiveUTF8('\x48\x45\x4c\x4c\x4f\x20\x57\x4f\x52\x4c\x44',['\x68\x65\x6c\x6c\x6f','\x77\x6f\x72\x6c\x64']);
```
Result:
```response
1
```
## multiSearchAny
Returns 1, if at least one string needle<sub>i</sub> matches the string `haystack` and 0 otherwise.
Functions `multiSearchAnyCaseInsensitive`, `multiSearchAnyUTF8` and `multiSearchAnyCaseInsensitiveUTF8` provide case-insensitive and/or UTF-8 variants of this function.
Functions [`multiSearchAnyCaseInsensitive`](#multiSearchAnyCaseInsensitive), [`multiSearchAnyUTF8`](#multiSearchAnyUTF8) and []`multiSearchAnyCaseInsensitiveUTF8`](#multiSearchAnyCaseInsensitiveUTF8) provide case-insensitive and/or UTF-8 variants of this function.
**Syntax**
```sql
multiSearchAny(haystack, \[needle<sub>1</sub>, needle<sub>2</sub>, …, needle<sub>n</sub>\])
multiSearchAny(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- 1, if there was at least one match.
- 0, if there was not at least one match.
**Example**
Query:
```sql
SELECT multiSearchAny('ClickHouse',['C','H']);
```
Result:
```response
1
```
## multiSearchAnyCaseInsensitive
Like [multiSearchAny](#multisearchany) but ignores case.
**Syntax**
```sql
multiSearchAnyCaseInsensitive(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — String in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — Substrings to be searched. [Array](../../sql-reference/data-types/array.md)
**Returned value**
- 1, if there was at least one case-insensitive match.
- 0, if there was not at least one case-insensitive match.
**Example**
Query:
```sql
SELECT multiSearchAnyCaseInsensitive('ClickHouse',['c','h']);
```
Result:
```response
1
```
## multiSearchAnyUTF8
Like [multiSearchAny](#multisearchany) but assumes `haystack` and the `needle` substrings are UTF-8 encoded strings.
*Syntax**
```sql
multiSearchAnyUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md).
**Returned value**
- 1, if there was at least one match.
- 0, if there was not at least one match.
**Example**
Given `ClickHouse` as a UTF-8 string, check if there are any `C` ('\x43') or `H` ('\x48') letters in the word.
Query:
```sql
SELECT multiSearchAnyUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x43','\x48']);
```
Result:
```response
1
```
## multiSearchAnyCaseInsensitiveUTF8
Like [multiSearchAnyUTF8](#multiSearchAnyUTF8) but ignores case.
*Syntax**
```sql
multiSearchAnyCaseInsensitiveUTF8(haystack, [needle1, needle2, ..., needleN])
```
**Parameters**
- `haystack` — UTF-8 string in which the search is performed. [String](../../sql-reference/syntax.md#syntax-string-literal).
- `needle` — UTF-8 substrings to be searched. [Array](../../sql-reference/data-types/array.md)
**Returned value**
- 1, if there was at least one case-insensitive match.
- 0, if there was not at least one case-insensitive match.
**Example**
Given `ClickHouse` as a UTF-8 string, check if there is any letter `h`(`\x68`) in the word, ignoring case.
Query:
```sql
SELECT multiSearchAnyCaseInsensitiveUTF8('\x43\x6c\x69\x63\x6b\x48\x6f\x75\x73\x65',['\x68']);
```
Result:
```response
1
```
## match {#match}

View File

@ -584,6 +584,278 @@ SELECT tupleConcat((1, 2), (3, 4), (true, false)) AS res
└──────────────────────┘
```
## tupleIntDiv
Does integer division of a tuple of numerators and a tuple of denominators, and returns a tuple of the quotients.
**Syntax**
```sql
tupleIntDiv(tuple_num, tuple_div)
```
**Parameters**
- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type.
- `tuple_div`: Tuple of divisor values. [Tuple](../data-types/tuple) of numeric type.
**Returned value**
- Tuple of the quotients of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of integer values.
**Implementation details**
- If either `tuple_num` or `tuple_div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor.
- An error will be thrown for division by 0.
**Examples**
Query:
``` sql
SELECT tupleIntDiv((15, 10, 5), (5, 5, 5));
```
Result:
``` text
┌─tupleIntDiv((15, 10, 5), (5, 5, 5))─┐
│ (3,2,1) │
└─────────────────────────────────────┘
```
Query:
``` sql
SELECT tupleIntDiv((15, 10, 5), (5.5, 5.5, 5.5));
```
Result:
``` text
┌─tupleIntDiv((15, 10, 5), (5.5, 5.5, 5.5))─┐
│ (2,1,0) │
└───────────────────────────────────────────┘
```
## tupleIntDivOrZero
Like [tupleIntDiv](#tupleintdiv) it does integer division of a tuple of numerators and a tuple of denominators, and returns a tuple of the quotients. It does not throw an error for 0 divisors, but rather returns the quotient as 0.
**Syntax**
```sql
tupleIntDivOrZero(tuple_num, tuple_div)
```
- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type.
- `tuple_div`: Tuple of divisor values. [Tuple](../data-types/tuple) of numeric type.
**Returned value**
- Tuple of the quotients of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of integer values.
- Returns 0 for quotients where the divisor is 0.
**Implementation details**
- If either `tuple_num` or `tuple_div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor as in [tupleIntDiv](#tupleintdiv).
**Examples**
Query:
``` sql
SELECT tupleIntDivOrZero((5, 10, 15), (0, 0, 0));
```
Result:
``` text
┌─tupleIntDivOrZero((5, 10, 15), (0, 0, 0))─┐
│ (0,0,0) │
└───────────────────────────────────────────┘
```
## tupleIntDivByNumber
Does integer division of a tuple of numerators by a given denominator, and returns a tuple of the quotients.
**Syntax**
```sql
tupleIntDivByNumber(tuple_num, div)
```
**Parameters**
- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type.
- `div`: The divisor value. [Numeric](../data-types/int-uint.md) type.
**Returned value**
- Tuple of the quotients of `tuple_num` and `div`. [Tuple](../data-types/tuple) of integer values.
**Implementation details**
- If either `tuple_num` or `div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor.
- An error will be thrown for division by 0.
**Examples**
Query:
``` sql
SELECT tupleIntDivByNumber((15, 10, 5), 5);
```
Result:
``` text
┌─tupleIntDivByNumber((15, 10, 5), 5)─┐
│ (3,2,1) │
└─────────────────────────────────────┘
```
Query:
``` sql
SELECT tupleIntDivByNumber((15.2, 10.7, 5.5), 5.8);
```
Result:
``` text
┌─tupleIntDivByNumber((15.2, 10.7, 5.5), 5.8)─┐
│ (2,1,0) │
└─────────────────────────────────────────────┘
```
## tupleIntDivOrZeroByNumber
Like [tupleIntDivByNumber](#tupleintdivbynumber) it does integer division of a tuple of numerators by a given denominator, and returns a tuple of the quotients. It does not throw an error for 0 divisors, but rather returns the quotient as 0.
**Syntax**
```sql
tupleIntDivOrZeroByNumber(tuple_num, div)
```
**Parameters**
- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type.
- `div`: The divisor value. [Numeric](../data-types/int-uint.md) type.
**Returned value**
- Tuple of the quotients of `tuple_num` and `div`. [Tuple](../data-types/tuple) of integer values.
- Returns 0 for quotients where the divisor is 0.
**Implementation details**
- If either `tuple_num` or `div` contain non-integer values then the result is calculated by rounding to the nearest integer for each non-integer numerator or divisor as in [tupleIntDivByNumber](#tupleintdivbynumber).
**Examples**
Query:
``` sql
SELECT tupleIntDivOrZeroByNumber((15, 10, 5), 5);
```
Result:
``` text
┌─tupleIntDivOrZeroByNumber((15, 10, 5), 5)─┐
│ (3,2,1) │
└───────────────────────────────────────────┘
```
Query:
``` sql
SELECT tupleIntDivOrZeroByNumber((15, 10, 5), 0)
```
Result:
``` text
┌─tupleIntDivOrZeroByNumber((15, 10, 5), 0)─┐
│ (0,0,0) │
└───────────────────────────────────────────┘
```
## tupleModulo
Returns a tuple of the moduli (remainders) of division operations of two tuples.
**Syntax**
```sql
tupleModulo(tuple_num, tuple_mod)
```
**Parameters**
- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type.
- `tuple_div`: Tuple of modulus values. [Tuple](../data-types/tuple) of numeric type.
**Returned value**
- Tuple of the remainders of division of `tuple_num` and `tuple_div`. [Tuple](../data-types/tuple) of non-zero integer values.
- An error is thrown for division by zero.
**Examples**
Query:
``` sql
SELECT tupleModulo((15, 10, 5), (5, 3, 2));
```
Result:
``` text
┌─tupleModulo((15, 10, 5), (5, 3, 2))─┐
│ (0,1,1) │
└─────────────────────────────────────┘
```
## tupleModuloByNumber
Returns a tuple of the moduli (remainders) of division operations of a tuple and a given divisor.
**Syntax**
```sql
tupleModuloByNumber(tuple_num, div)
```
**Parameters**
- `tuple_num`: Tuple of numerator values. [Tuple](../data-types/tuple) of numeric type.
- `div`: The divisor value. [Numeric](../data-types/int-uint.md) type.
**Returned value**
- Tuple of the remainders of division of `tuple_num` and `div`. [Tuple](../data-types/tuple) of non-zero integer values.
- An error is thrown for division by zero.
**Examples**
Query:
``` sql
SELECT tupleModuloByNumber((15, 10, 5), 2);
```
Result:
``` text
┌─tupleModuloByNumber((15, 10, 5), 2)─┐
│ (1,0,1) │
└─────────────────────────────────────┘
```
## Distance functions
All supported functions are described in [distance functions documentation](../../sql-reference/functions/distance-functions.md).

View File

@ -20,11 +20,10 @@ DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] [SYNC]
## DROP TABLE
Deletes the table.
In case when `IF EMPTY` clause is specified server will check if table is empty only on replica that received initial query.
Deletes one or more tables.
:::tip
Also see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md)
To undo the deletion of a table, please see [UNDROP TABLE](/docs/en/sql-reference/statements/undrop.md)
:::
Syntax:
@ -33,7 +32,9 @@ Syntax:
DROP [TEMPORARY] TABLE [IF EXISTS] [IF EMPTY] [db1.]name_1[, [db2.]name_2, ...] [ON CLUSTER cluster] [SYNC]
```
Note that deleting multiple tables at the same time is a non-atomic deletion. If a table fails to be deleted, subsequent tables will not be deleted.
Limitations:
- If the clause `IF EMPTY` is specified, the server checks the emptiness of the table only on the replica which received the query.
- Deleting multiple tables at once is not an atomic operation, i.e. if the deletion of a table fails, subsequent tables will not be deleted.
## DROP DICTIONARY

View File

@ -12,25 +12,23 @@ Some of the calculations that you can do are similar to those that can be done w
ClickHouse supports the standard grammar for defining windows and window functions. The table below indicates whether a feature is currently supported.
| Feature | Support or workaround |
| Feature | Supported? |
|------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| ad hoc window specification (`count(*) over (partition by id order by time desc)`) | supported |
| expressions involving window functions, e.g. `(count(*) over ()) / 2)` | supported |
| `WINDOW` clause (`select ... from table window w as (partition by id)`) | supported |
| `ROWS` frame | supported |
| `RANGE` frame | supported, the default |
| `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | not supported, specify the number of seconds instead (`RANGE` works with any numeric type). |
| `GROUPS` frame | not supported |
| Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | all aggregate functions are supported |
| `rank()`, `dense_rank()`, `row_number()` | supported |
| `lag/lead(value, offset)` | Not supported. Workarounds: |
| | 1) replace with `any(value) over (.... rows between <offset> preceding and <offset> preceding)`, or `following` for `lead` |
| | 2) use `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` |
| ntile(buckets) | Supported. Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). |
| ad hoc window specification (`count(*) over (partition by id order by time desc)`) | ✅ |
| expressions involving window functions, e.g. `(count(*) over ()) / 2)` | ✅ |
| `WINDOW` clause (`select ... from table window w as (partition by id)`) | ✅ |
| `ROWS` frame | ✅ |
| `RANGE` frame | ✅ (the default) |
| `INTERVAL` syntax for `DateTime` `RANGE OFFSET` frame | ❌ (specify the number of seconds instead (`RANGE` works with any numeric type).) |
| `GROUPS` frame | ❌ |
| Calculating aggregate functions over a frame (`sum(value) over (order by time)`) | ✅ (All aggregate functions are supported) |
| `rank()`, `dense_rank()`, `row_number()` | ✅ |
| `lag/lead(value, offset)` | ❌ <br/> You can use one of the following workarounds:<br/> 1) `any(value) over (.... rows between <offset> preceding and <offset> preceding)`, or `following` for `lead` <br/> 2) `lagInFrame/leadInFrame`, which are analogous, but respect the window frame. To get behavior identical to `lag/lead`, use `rows between unbounded preceding and unbounded following` |
| ntile(buckets) | ✅ <br/> Specify window like, (partition by x order by y rows between unbounded preceding and unrounded following). |
## ClickHouse-specific Window Functions
There are also the following window function that's specific to ClickHouse:
There is also the following ClickHouse specific window function:
### nonNegativeDerivative(metric_column, timestamp_column[, INTERVAL X UNITS])
@ -89,6 +87,102 @@ These functions can be used only as a window function.
Let's have a look at some examples of how window functions can be used.
### Numbering rows
```sql
CREATE TABLE salaries
(
`team` String,
`player` String,
`salary` UInt32,
`position` String
)
Engine = Memory;
INSERT INTO salaries FORMAT Values
('Port Elizabeth Barbarians', 'Gary Chen', 195000, 'F'),
('New Coreystad Archdukes', 'Charles Juarez', 190000, 'F'),
('Port Elizabeth Barbarians', 'Michael Stanley', 150000, 'D'),
('New Coreystad Archdukes', 'Scott Harrison', 150000, 'D'),
('Port Elizabeth Barbarians', 'Robert George', 195000, 'M');
```
```sql
SELECT player, salary,
row_number() OVER (ORDER BY salary) AS row
FROM salaries;
```
```text
┌─player──────────┬─salary─┬─row─┐
│ Michael Stanley │ 150000 │ 1 │
│ Scott Harrison │ 150000 │ 2 │
│ Charles Juarez │ 190000 │ 3 │
│ Gary Chen │ 195000 │ 4 │
│ Robert George │ 195000 │ 5 │
└─────────────────┴────────┴─────┘
```
```sql
SELECT player, salary,
row_number() OVER (ORDER BY salary) AS row,
rank() OVER (ORDER BY salary) AS rank,
dense_rank() OVER (ORDER BY salary) AS denseRank
FROM salaries;
```
```text
┌─player──────────┬─salary─┬─row─┬─rank─┬─denseRank─┐
│ Michael Stanley │ 150000 │ 1 │ 1 │ 1 │
│ Scott Harrison │ 150000 │ 2 │ 1 │ 1 │
│ Charles Juarez │ 190000 │ 3 │ 3 │ 2 │
│ Gary Chen │ 195000 │ 4 │ 4 │ 3 │
│ Robert George │ 195000 │ 5 │ 4 │ 3 │
└─────────────────┴────────┴─────┴──────┴───────────┘
```
### Aggregation functions
Compare each player's salary to the average for their team.
```sql
SELECT player, salary, team,
avg(salary) OVER (PARTITION BY team) AS teamAvg,
salary - teamAvg AS diff
FROM salaries;
```
```text
┌─player──────────┬─salary─┬─team──────────────────────┬─teamAvg─┬───diff─┐
│ Charles Juarez │ 190000 │ New Coreystad Archdukes │ 170000 │ 20000 │
│ Scott Harrison │ 150000 │ New Coreystad Archdukes │ 170000 │ -20000 │
│ Gary Chen │ 195000 │ Port Elizabeth Barbarians │ 180000 │ 15000 │
│ Michael Stanley │ 150000 │ Port Elizabeth Barbarians │ 180000 │ -30000 │
│ Robert George │ 195000 │ Port Elizabeth Barbarians │ 180000 │ 15000 │
└─────────────────┴────────┴───────────────────────────┴─────────┴────────┘
```
Compare each player's salary to the maximum for their team.
```sql
SELECT player, salary, team,
max(salary) OVER (PARTITION BY team) AS teamAvg,
salary - teamAvg AS diff
FROM salaries;
```
```text
┌─player──────────┬─salary─┬─team──────────────────────┬─teamAvg─┬───diff─┐
│ Charles Juarez │ 190000 │ New Coreystad Archdukes │ 190000 │ 0 │
│ Scott Harrison │ 150000 │ New Coreystad Archdukes │ 190000 │ -40000 │
│ Gary Chen │ 195000 │ Port Elizabeth Barbarians │ 195000 │ 0 │
│ Michael Stanley │ 150000 │ Port Elizabeth Barbarians │ 195000 │ -45000 │
│ Robert George │ 195000 │ Port Elizabeth Barbarians │ 195000 │ 0 │
└─────────────────┴────────┴───────────────────────────┴─────────┴────────┘
```
### Partitioning by column
```sql
CREATE TABLE wf_partition
(
@ -120,6 +214,8 @@ ORDER BY
└──────────┴───────┴───────┴──────────────┘
```
### Frame bounding
```sql
CREATE TABLE wf_frame
(
@ -131,14 +227,19 @@ ENGINE = Memory;
INSERT INTO wf_frame FORMAT Values
(1,1,1), (1,2,2), (1,3,3), (1,4,4), (1,5,5);
```
-- frame is bounded by bounds of a partition (BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)
```sql
-- Frame is bounded by bounds of a partition (BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)
SELECT
part_key,
value,
order,
groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC
Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING) AS frame_values
groupArray(value) OVER (
PARTITION BY part_key
ORDER BY order ASC
Rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
) AS frame_values
FROM wf_frame
ORDER BY
part_key ASC,
@ -151,7 +252,9 @@ ORDER BY
│ 1 │ 4 │ 4 │ [1,2,3,4,5] │
│ 1 │ 5 │ 5 │ [1,2,3,4,5] │
└──────────┴───────┴───────┴──────────────┘
```
```sql
-- short form - no bound expression, no order by
SELECT
part_key,
@ -169,14 +272,19 @@ ORDER BY
│ 1 │ 4 │ 4 │ [1,2,3,4,5] │
│ 1 │ 5 │ 5 │ [1,2,3,4,5] │
└──────────┴───────┴───────┴──────────────┘
```
-- frame is bounded by the beggining of a partition and the current row
```sql
-- frame is bounded by the beginning of a partition and the current row
SELECT
part_key,
value,
order,
groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC
Rows BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW) AS frame_values
groupArray(value) OVER (
PARTITION BY part_key
ORDER BY order ASC
Rows BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW
) AS frame_values
FROM wf_frame
ORDER BY
part_key ASC,
@ -189,8 +297,10 @@ ORDER BY
│ 1 │ 4 │ 4 │ [1,2,3,4] │
│ 1 │ 5 │ 5 │ [1,2,3,4,5] │
└──────────┴───────┴───────┴──────────────┘
```
-- short form (frame is bounded by the beggining of a partition and the current row)
```sql
-- short form (frame is bounded by the beginning of a partition and the current row)
SELECT
part_key,
value,
@ -207,8 +317,10 @@ ORDER BY
│ 1 │ 4 │ 4 │ [1,2,3,4] │
│ 1 │ 5 │ 5 │ [1,2,3,4,5] │
└──────────┴───────┴───────┴──────────────┘
```
-- frame is bounded by the beggining of a partition and the current row, but order is backward
```sql
-- frame is bounded by the beginning of a partition and the current row, but order is backward
SELECT
part_key,
value,
@ -225,14 +337,19 @@ ORDER BY
│ 1 │ 4 │ 4 │ [5,4] │
│ 1 │ 5 │ 5 │ [5] │
└──────────┴───────┴───────┴──────────────┘
```
```sql
-- sliding frame - 1 PRECEDING ROW AND CURRENT ROW
SELECT
part_key,
value,
order,
groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC
Rows BETWEEN 1 PRECEDING AND CURRENT ROW) AS frame_values
groupArray(value) OVER (
PARTITION BY part_key
ORDER BY order ASC
Rows BETWEEN 1 PRECEDING AND CURRENT ROW
) AS frame_values
FROM wf_frame
ORDER BY
part_key ASC,
@ -245,14 +362,19 @@ ORDER BY
│ 1 │ 4 │ 4 │ [3,4] │
│ 1 │ 5 │ 5 │ [4,5] │
└──────────┴───────┴───────┴──────────────┘
```
```sql
-- sliding frame - Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING
SELECT
part_key,
value,
order,
groupArray(value) OVER (PARTITION BY part_key ORDER BY order ASC
Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING) AS frame_values
groupArray(value) OVER (
PARTITION BY part_key
ORDER BY order ASC
Rows BETWEEN 1 PRECEDING AND UNBOUNDED FOLLOWING
) AS frame_values
FROM wf_frame
ORDER BY
part_key ASC,
@ -264,7 +386,9 @@ ORDER BY
│ 1 │ 4 │ 4 │ [3,4,5] │
│ 1 │ 5 │ 5 │ [4,5] │
└──────────┴───────┴───────┴──────────────┘
```
```sql
-- row_number does not respect the frame, so rn_1 = rn_2 = rn_3 != rn_4
SELECT
part_key,
@ -278,8 +402,11 @@ SELECT
FROM wf_frame
WINDOW
w1 AS (PARTITION BY part_key ORDER BY order DESC),
w2 AS (PARTITION BY part_key ORDER BY order DESC
Rows BETWEEN 1 PRECEDING AND CURRENT ROW)
w2 AS (
PARTITION BY part_key
ORDER BY order DESC
Rows BETWEEN 1 PRECEDING AND CURRENT ROW
)
ORDER BY
part_key ASC,
value ASC;
@ -290,7 +417,9 @@ ORDER BY
│ 1 │ 4 │ 4 │ [5,4] │ 2 │ 2 │ 2 │ 2 │
│ 1 │ 5 │ 5 │ [5] │ 1 │ 1 │ 1 │ 1 │
└──────────┴───────┴───────┴──────────────┴──────┴──────┴──────┴──────┘
```
```sql
-- first_value and last_value respect the frame
SELECT
groupArray(value) OVER w1 AS frame_values_1,
@ -313,7 +442,9 @@ ORDER BY
│ [1,2,3,4] │ 1 │ 4 │ [3,4] │ 3 │ 4 │
│ [1,2,3,4,5] │ 1 │ 5 │ [4,5] │ 4 │ 5 │
└────────────────┴───────────────┴──────────────┴────────────────┴───────────────┴──────────────┘
```
```sql
-- second value within the frame
SELECT
groupArray(value) OVER w1 AS frame_values_1,
@ -330,7 +461,9 @@ ORDER BY
│ [1,2,3,4] │ 2 │
│ [2,3,4,5] │ 3 │
└────────────────┴──────────────┘
```
```sql
-- second value within the frame + Null for missing values
SELECT
groupArray(value) OVER w1 AS frame_values_1,
@ -351,7 +484,9 @@ ORDER BY
## Real world examples
### Maximum/total salary per department.
The following examples solve common real-world problems.
### Maximum/total salary per department
```sql
CREATE TABLE employees
@ -369,7 +504,9 @@ INSERT INTO employees FORMAT Values
('IT', 'Tim', 200),
('IT', 'Anna', 300),
('IT', 'Elen', 500);
```
```sql
SELECT
department,
employee_name AS emp,
@ -386,8 +523,10 @@ FROM
max(salary) OVER wndw AS max_salary_per_dep,
sum(salary) OVER wndw AS total_salary_per_dep
FROM employees
WINDOW wndw AS (PARTITION BY department
rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING)
WINDOW wndw AS (
PARTITION BY department
rows BETWEEN UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING
)
ORDER BY
department ASC,
employee_name ASC
@ -403,7 +542,7 @@ FROM
└────────────┴──────┴────────┴────────────────────┴──────────────────────┴──────────────────┘
```
### Cumulative sum.
### Cumulative sum
```sql
CREATE TABLE warehouse
@ -421,7 +560,9 @@ INSERT INTO warehouse VALUES
('sku1', '2020-01-01', 1),
('sku1', '2020-02-01', 1),
('sku1', '2020-03-01', 1);
```
```sql
SELECT
item,
ts,
@ -461,13 +602,18 @@ insert into sensors values('cpu_temp', '2020-01-01 00:00:00', 87),
('cpu_temp', '2020-01-01 00:00:05', 87),
('cpu_temp', '2020-01-01 00:00:06', 87),
('cpu_temp', '2020-01-01 00:00:07', 87);
```
```sql
SELECT
metric,
ts,
value,
avg(value) OVER
(PARTITION BY metric ORDER BY ts ASC Rows BETWEEN 2 PRECEDING AND CURRENT ROW)
AS moving_avg_temp
avg(value) OVER (
PARTITION BY metric
ORDER BY ts ASC
Rows BETWEEN 2 PRECEDING AND CURRENT ROW
) AS moving_avg_temp
FROM sensors
ORDER BY
metric ASC,
@ -536,7 +682,9 @@ insert into sensors values('ambient_temp', '2020-01-01 00:00:00', 16),
('ambient_temp', '2020-03-01 12:00:00', 16),
('ambient_temp', '2020-03-01 12:00:00', 16),
('ambient_temp', '2020-03-01 12:00:00', 16);
```
```sql
SELECT
metric,
ts,

View File

@ -1,28 +0,0 @@
---
slug: /ru/sql-reference/data-types/multiword-types
sidebar_position: 61
sidebar_label: Составные типы
---
# Составные типы {#multiword-types}
При создании таблиц вы можете использовать типы данных с названием, состоящим из нескольких слов. Такие названия поддерживаются для лучшей совместимости с SQL.
## Поддержка составных типов {#multiword-types-support}
| Составные типы | Обычные типы |
|-------------------------------------|-----------------------------------------------------------|
| DOUBLE PRECISION | [Float64](../../sql-reference/data-types/float.md) |
| CHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NCHAR LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NCHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR VARYING | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHARACTER | [String](../../sql-reference/data-types/string.md) |
| NATIONAL CHAR | [String](../../sql-reference/data-types/string.md) |
| BINARY LARGE OBJECT | [String](../../sql-reference/data-types/string.md) |
| BINARY VARYING | [String](../../sql-reference/data-types/string.md) |

View File

@ -1,10 +0,0 @@
---
slug: /zh/sql-reference/data-types/multiword-types
sidebar_position: 61
sidebar_label: Multiword Type Names
title: "Multiword Types"
---
import Content from '@site/docs/en/sql-reference/data-types/multiword-types.md';
<Content />

View File

@ -3,7 +3,7 @@
function _clickhouse_get_utils()
{
local cmd=$1 && shift
"$cmd" --help |& awk '/^clickhouse.*args/ { print $2 }'
"$cmd" help |& awk '/^clickhouse.*args/ { print $2 }'
}
function _complete_for_clickhouse_entrypoint_bin()

View File

@ -934,8 +934,8 @@ void Client::addOptions(OptionsDescription & options_description)
("user,u", po::value<std::string>()->default_value("default"), "user")
("password", po::value<std::string>(), "password")
("ask-password", "ask-password")
("ssh-key-file", po::value<std::string>(), "File containing ssh private key needed for authentication. If not set does password authentication.")
("ssh-key-passphrase", po::value<std::string>(), "Passphrase for imported ssh key.")
("ssh-key-file", po::value<std::string>(), "File containing the SSH private key for authenticate with the server.")
("ssh-key-passphrase", po::value<std::string>(), "Passphrase for the SSH private key specified by --ssh-key-file.")
("quota_key", po::value<std::string>(), "A string to differentiate quotas when the user have keyed quotas configured on server")
("max_client_network_bandwidth", po::value<int>(), "the maximum speed of data exchange over the network for the client in bytes per second.")

View File

@ -166,7 +166,7 @@ int DisksApp::main(const std::vector<String> & /*args*/)
{
String config_path = config().getString("config-file", getDefaultConfigFileName());
ConfigProcessor config_processor(config_path, false, false);
config_processor.setConfigPath(fs::path(config_path).parent_path());
ConfigProcessor::setConfigPath(fs::path(config_path).parent_path());
auto loaded_config = config_processor.loadConfig();
config().add(loaded_config.configuration.duplicate(), false, false);
}

View File

@ -368,7 +368,7 @@ int KeeperClient::main(const std::vector<String> & /* args */)
DB::ConfigProcessor config_processor(config().getString("config-file", "config.xml"));
/// This will handle a situation when clickhouse is running on the embedded config, but config.d folder is also present.
config_processor.registerEmbeddedConfig("config.xml", "<clickhouse/>");
ConfigProcessor::registerEmbeddedConfig("config.xml", "<clickhouse/>");
auto clickhouse_config = config_processor.loadConfig();
Poco::Util::AbstractConfiguration::Keys keys;

View File

@ -122,7 +122,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
{
const auto config_path = config().getString("config-file", "config.xml");
ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(fs::path(config_path).parent_path());
ConfigProcessor::setConfigPath(fs::path(config_path).parent_path());
auto loaded_config = config_processor.loadConfig();
config().add(loaded_config.configuration.duplicate(), PRIO_DEFAULT, false);
}

View File

@ -487,7 +487,7 @@ int main(int argc_, char ** argv_)
/// Interpret binary without argument or with arguments starts with dash
/// ('-') as clickhouse-local for better usability:
///
/// clickhouse # dumps help
/// clickhouse help # dumps help
/// clickhouse -q 'select 1' # use local
/// clickhouse # spawn local
/// clickhouse local # spawn local

View File

@ -4,11 +4,12 @@
#include <Access/ExternalAuthenticators.h>
#include <Access/LDAPClient.h>
#include <Access/GSSAcceptor.h>
#include <Common/Exception.h>
#include <Poco/SHA1Engine.h>
#include <Common/Exception.h>
#include <Common/SSHWrapper.h>
#include <Common/typeid_cast.h>
#include <Common/SSH/Wrappers.h>
#include "config.h"
namespace DB
{
@ -74,7 +75,7 @@ namespace
}
#if USE_SSH
bool checkSshSignature(const std::vector<ssh::SSHKey> & keys, std::string_view signature, std::string_view original)
bool checkSshSignature(const std::vector<SSHKey> & keys, std::string_view signature, std::string_view original)
{
for (const auto & key: keys)
if (key.isPublic() && key.verifySignature(signature, original))
@ -114,7 +115,11 @@ bool Authentication::areCredentialsValid(
throw Authentication::Require<BasicCredentials>("ClickHouse X.509 Authentication");
case AuthenticationType::SSH_KEY:
throw Authentication::Require<SshCredentials>("Ssh Keys Authentication");
#if USE_SSH
throw Authentication::Require<SshCredentials>("SSH Keys Authentication");
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
case AuthenticationType::MAX:
break;
@ -145,7 +150,11 @@ bool Authentication::areCredentialsValid(
throw Authentication::Require<BasicCredentials>("ClickHouse X.509 Authentication");
case AuthenticationType::SSH_KEY:
throw Authentication::Require<SshCredentials>("Ssh Keys Authentication");
#if USE_SSH
throw Authentication::Require<SshCredentials>("SSH Keys Authentication");
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
case AuthenticationType::MAX:
break;
@ -178,7 +187,11 @@ bool Authentication::areCredentialsValid(
throw Authentication::Require<BasicCredentials>("ClickHouse X.509 Authentication");
case AuthenticationType::SSH_KEY:
throw Authentication::Require<SshCredentials>("Ssh Keys Authentication");
#if USE_SSH
throw Authentication::Require<SshCredentials>("SSH Keys Authentication");
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
case AuthenticationType::BCRYPT_PASSWORD:
return checkPasswordBcrypt(basic_credentials->getPassword(), auth_data.getPasswordHashBinary());
@ -216,13 +229,18 @@ bool Authentication::areCredentialsValid(
return auth_data.getSSLCertificateCommonNames().contains(ssl_certificate_credentials->getCommonName());
case AuthenticationType::SSH_KEY:
throw Authentication::Require<SshCredentials>("Ssh Keys Authentication");
#if USE_SSH
throw Authentication::Require<SshCredentials>("SSH Keys Authentication");
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
case AuthenticationType::MAX:
break;
}
}
#if USE_SSH
if (const auto * ssh_credentials = typeid_cast<const SshCredentials *>(&credentials))
{
switch (auth_data.getType())
@ -243,15 +261,12 @@ bool Authentication::areCredentialsValid(
throw Authentication::Require<SSLCertificateCredentials>("ClickHouse X.509 Authentication");
case AuthenticationType::SSH_KEY:
#if USE_SSH
return checkSshSignature(auth_data.getSSHKeys(), ssh_credentials->getSignature(), ssh_credentials->getOriginal());
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL");
#endif
case AuthenticationType::MAX:
break;
}
}
#endif
if ([[maybe_unused]] const auto * always_allow_credentials = typeid_cast<const AlwaysAllowCredentials *>(&credentials))
return true;

View File

@ -105,7 +105,10 @@ bool operator ==(const AuthenticationData & lhs, const AuthenticationData & rhs)
return (lhs.type == rhs.type) && (lhs.password_hash == rhs.password_hash)
&& (lhs.ldap_server_name == rhs.ldap_server_name) && (lhs.kerberos_realm == rhs.kerberos_realm)
&& (lhs.ssl_certificate_common_names == rhs.ssl_certificate_common_names)
&& (lhs.ssh_keys == rhs.ssh_keys) && (lhs.http_auth_scheme == rhs.http_auth_scheme)
#if USE_SSH
&& (lhs.ssh_keys == rhs.ssh_keys)
#endif
&& (lhs.http_auth_scheme == rhs.http_auth_scheme)
&& (lhs.http_auth_server_name == rhs.http_auth_server_name);
}
@ -326,7 +329,7 @@ std::shared_ptr<ASTAuthenticationData> AuthenticationData::toAST() const
break;
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL");
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
}
case AuthenticationType::HTTP:
@ -355,7 +358,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
{
#if USE_SSH
AuthenticationData auth_data(*query.type);
std::vector<ssh::SSHKey> keys;
std::vector<SSHKey> keys;
size_t args_size = query.children.size();
for (size_t i = 0; i < args_size; ++i)
@ -366,7 +369,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
try
{
keys.emplace_back(ssh::SSHKeyFactory::makePublicFromBase64(key_base64, type));
keys.emplace_back(SSHKeyFactory::makePublicKeyFromBase64(key_base64, type));
}
catch (const std::invalid_argument &)
{
@ -377,7 +380,7 @@ AuthenticationData AuthenticationData::fromAST(const ASTAuthenticationData & que
auth_data.setSSHKeys(std::move(keys));
return auth_data;
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL");
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
}

View File

@ -2,14 +2,16 @@
#include <Access/Common/AuthenticationType.h>
#include <Access/Common/HTTPAuthenticationScheme.h>
#include <Common/SSHWrapper.h>
#include <Interpreters/Context_fwd.h>
#include <Parsers/Access/ASTAuthenticationData.h>
#include <Common/SSH/Wrappers.h>
#include <vector>
#include <base/types.h>
#include <boost/container/flat_set.hpp>
#include "config.h"
namespace DB
{
@ -59,8 +61,10 @@ public:
const boost::container::flat_set<String> & getSSLCertificateCommonNames() const { return ssl_certificate_common_names; }
void setSSLCertificateCommonNames(boost::container::flat_set<String> common_names_);
const std::vector<ssh::SSHKey> & getSSHKeys() const { return ssh_keys; }
void setSSHKeys(std::vector<ssh::SSHKey> && ssh_keys_) { ssh_keys = std::forward<std::vector<ssh::SSHKey>>(ssh_keys_); }
#if USE_SSH
const std::vector<SSHKey> & getSSHKeys() const { return ssh_keys; }
void setSSHKeys(std::vector<SSHKey> && ssh_keys_) { ssh_keys = std::forward<std::vector<SSHKey>>(ssh_keys_); }
#endif
HTTPAuthenticationScheme getHTTPAuthenticationScheme() const { return http_auth_scheme; }
void setHTTPAuthenticationScheme(HTTPAuthenticationScheme scheme) { http_auth_scheme = scheme; }
@ -94,7 +98,9 @@ private:
String kerberos_realm;
boost::container::flat_set<String> ssl_certificate_common_names;
String salt;
std::vector<ssh::SSHKey> ssh_keys;
#if USE_SSH
std::vector<SSHKey> ssh_keys;
#endif
/// HTTP authentication properties
String http_auth_server_name;
HTTPAuthenticationScheme http_auth_scheme = HTTPAuthenticationScheme::BASIC;

View File

@ -34,8 +34,8 @@ enum class AuthenticationType
/// Password is encrypted in bcrypt hash.
BCRYPT_PASSWORD,
/// Server sends a random string named `challenge` which client needs to encrypt with private key.
/// The check is performed on server side by decrypting the data and comparing with the original string.
/// Server sends a random string named `challenge` to the client. The client encrypts it with its SSH private key.
/// The server decrypts the result using the SSH public key registered for the user and compares with the original string.
SSH_KEY,
/// Authentication through HTTP protocol

View File

@ -3,6 +3,7 @@
#include <base/types.h>
#include <memory>
#include "config.h"
namespace DB
{
@ -86,10 +87,11 @@ class MySQLNative41Credentials : public CredentialsWithScramble
using CredentialsWithScramble::CredentialsWithScramble;
};
#if USE_SSH
class SshCredentials : public Credentials
{
public:
explicit SshCredentials(const String& user_name_, const String& signature_, const String& original_)
SshCredentials(const String & user_name_, const String & signature_, const String & original_)
: Credentials(user_name_), signature(signature_), original(original_)
{
is_ready = true;
@ -117,5 +119,6 @@ private:
String signature;
String original;
};
#endif
}

View File

@ -31,7 +31,7 @@ void User::setName(const String & name_)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User name is empty");
if (name_ == EncodedUserInfo::USER_INTERSERVER_MARKER)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User name '{}' is reserved", name_);
if (startsWith(name_, EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER))
if (name_.starts_with(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "User name '{}' is reserved", name_);
name = name_;
}

View File

@ -1,6 +1,5 @@
#include <Access/UsersConfigAccessStorage.h>
#include <Access/Quota.h>
#include <Common/SSH/Wrappers.h>
#include <Access/RowPolicy.h>
#include <Access/User.h>
#include <Access/Role.h>
@ -10,6 +9,7 @@
#include <Access/AccessChangesNotifier.h>
#include <Dictionaries/IDictionary.h>
#include <Common/Config/ConfigReloader.h>
#include <Common/SSHWrapper.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h>
#include <Common/transformEndianness.h>
@ -214,7 +214,7 @@ namespace
Poco::Util::AbstractConfiguration::Keys entries;
config.keys(ssh_keys_config, entries);
std::vector<ssh::SSHKey> keys;
std::vector<SSHKey> keys;
for (const String& entry : entries)
{
const auto conf_pref = ssh_keys_config + "." + entry + ".";
@ -237,7 +237,7 @@ namespace
try
{
keys.emplace_back(ssh::SSHKeyFactory::makePublicFromBase64(base64_key, type));
keys.emplace_back(SSHKeyFactory::makePublicKeyFromBase64(base64_key, type));
}
catch (const std::invalid_argument &)
{
@ -249,7 +249,7 @@ namespace
}
user->auth_data.setSSHKeys(std::move(keys));
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL");
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
}
else if (has_http_auth)

View File

@ -54,30 +54,30 @@ public:
{
const auto & value = columns[0]->getFloat64(row_num);
const auto & time = columns[1]->getFloat64(row_num);
this->data(place).add(value, time, half_decay);
data(place).add(value, time, half_decay);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena *) const override
{
this->data(place).merge(this->data(rhs), half_decay);
data(place).merge(data(rhs), half_decay);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
writeBinary(this->data(place).value, buf);
writeBinary(this->data(place).time, buf);
writeBinary(data(place).value, buf);
writeBinary(data(place).time, buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena *) const override
{
readBinary(this->data(place).value, buf);
readBinary(this->data(place).time, buf);
readBinary(data(place).value, buf);
readBinary(data(place).time, buf);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
auto & column = assert_cast<ColumnVector<Float64> &>(to);
column.getData().push_back(this->data(place).get(half_decay));
column.getData().push_back(data(place).get(half_decay));
}
};

View File

@ -293,32 +293,32 @@ public:
Float64 value = columns[0]->getFloat64(row_num);
UInt8 is_second = columns[1]->getUInt(row_num);
if (is_second)
this->data(place).addY(value, arena);
data(place).addY(value, arena);
else
this->data(place).addX(value, arena);
data(place).addX(value, arena);
}
void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
this->data(place).merge(this->data(rhs), arena);
data(place).merge(data(rhs), arena);
}
void serialize(ConstAggregateDataPtr __restrict place, WriteBuffer & buf, std::optional<size_t> /* version */) const override
{
this->data(place).write(buf);
data(place).write(buf);
}
void deserialize(AggregateDataPtr __restrict place, ReadBuffer & buf, std::optional<size_t> /* version */, Arena * arena) const override
{
this->data(place).read(buf, arena);
data(place).read(buf, arena);
}
void insertResultInto(AggregateDataPtr __restrict place, IColumn & to, Arena *) const override
{
if (!this->data(place).size_x || !this->data(place).size_y)
if (!data(place).size_x || !data(place).size_y)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Aggregate function {} require both samples to be non empty", getName());
auto [d_statistic, p_value] = this->data(place).getResult(alternative, method);
auto [d_statistic, p_value] = data(place).getResult(alternative, method);
/// Because p-value is a probability.
p_value = std::min(1.0, std::max(0.0, p_value));

View File

@ -147,6 +147,8 @@ public:
negative_store->merge(other.negative_store.get());
}
/// NOLINTBEGIN(readability-static-accessed-through-instance)
void serialize(WriteBuffer& buf) const
{
// Write the mapping
@ -201,6 +203,8 @@ public:
count = static_cast<Float64>(negative_store->count + zero_count + store->count);
}
/// NOLINTEND(readability-static-accessed-through-instance)
private:
std::unique_ptr<DDSketchLogarithmicMapping> mapping;
std::unique_ptr<DDSketchDenseStore> store;

View File

@ -87,6 +87,8 @@ public:
count += other->count;
}
/// NOLINTBEGIN(readability-static-accessed-through-instance)
void serialize(WriteBuffer& buf) const
{
@ -179,6 +181,8 @@ public:
}
}
/// NOLINTEND(readability-static-accessed-through-instance)
private:
UInt32 chunk_size;
DDSketchEncoding enc;

View File

@ -5799,7 +5799,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
return result_projection_names;
}
FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters);
FunctionOverloadResolverPtr function = UserDefinedExecutableFunctionFactory::instance().tryGet(function_name, scope.context, parameters); /// NOLINT(readability-static-accessed-through-instance)
bool is_executable_udf = true;
IdentifierResolveScope::ResolvedFunctionsCache * function_cache = nullptr;
@ -5829,7 +5829,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
{
std::vector<std::string> possible_function_names;
auto function_names = UserDefinedExecutableFunctionFactory::instance().getRegisteredNames(scope.context);
auto function_names = UserDefinedExecutableFunctionFactory::instance().getRegisteredNames(scope.context); /// NOLINT(readability-static-accessed-through-instance)
possible_function_names.insert(possible_function_names.end(), function_names.begin(), function_names.end());
function_names = UserDefinedSQLFunctionFactory::instance().getAllRegisteredNames();
@ -5847,8 +5847,7 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
possible_function_names.push_back(name);
}
NamePrompter<2> name_prompter;
auto hints = name_prompter.getHints(function_name, possible_function_names);
auto hints = NamePrompter<2>::getHints(function_name, possible_function_names);
throw Exception(ErrorCodes::UNKNOWN_FUNCTION,
"Function with name '{}' does not exists. In scope {}{}",

View File

@ -85,7 +85,6 @@ add_headers_and_sources(clickhouse_common_io Common)
add_headers_and_sources(clickhouse_common_io Common/HashTable)
add_headers_and_sources(clickhouse_common_io Common/Scheduler)
add_headers_and_sources(clickhouse_common_io Common/Scheduler/Nodes)
add_headers_and_sources(clickhouse_common_io Common/SSH)
add_headers_and_sources(clickhouse_common_io IO)
add_headers_and_sources(clickhouse_common_io IO/Archives)
add_headers_and_sources(clickhouse_common_io IO/S3)
@ -99,7 +98,6 @@ add_headers_and_sources(clickhouse_compression Core)
#Included these specific files to avoid linking grpc
add_glob(clickhouse_compression_headers Server/ServerType.h)
add_glob(clickhouse_compression_sources Server/ServerType.cpp)
add_headers_and_sources(clickhouse_compression Common/SSH)
add_library(clickhouse_compression ${clickhouse_compression_headers} ${clickhouse_compression_sources})
@ -370,8 +368,7 @@ if (TARGET ch_contrib::crc32-vpmsum)
endif()
if (TARGET ch_contrib::ssh)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::ssh)
target_link_libraries(clickhouse_compression PUBLIC ch_contrib::ssh)
target_link_libraries(clickhouse_common_io PUBLIC ch_contrib::ssh)
endif()
dbms_target_link_libraries(PUBLIC ch_contrib::abseil_swiss_tables)

View File

@ -1964,7 +1964,7 @@ void ClientBase::processParsedSingleQuery(const String & full_query, const Strin
}
/// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately.
if (insert && (!insert->select || input_function) && !insert->watch && !is_async_insert_with_inlined_data)
if (insert && (!insert->select || input_function) && !is_async_insert_with_inlined_data)
{
if (input_function && insert->format.empty())
throw Exception(ErrorCodes::INVALID_USAGE_OF_INPUT, "FORMAT must be specified for function input()");

View File

@ -67,7 +67,7 @@ Connection::~Connection() = default;
Connection::Connection(const String & host_, UInt16 port_,
const String & default_database_,
const String & user_, const String & password_,
const ssh::SSHKey & ssh_private_key_,
[[maybe_unused]] const SSHKey & ssh_private_key_,
const String & quota_key_,
const String & cluster_,
const String & cluster_secret_,
@ -76,7 +76,9 @@ Connection::Connection(const String & host_, UInt16 port_,
Protocol::Secure secure_)
: host(host_), port(port_), default_database(default_database_)
, user(user_), password(password_)
#if USE_SSH
, ssh_private_key(ssh_private_key_)
#endif
, quota_key(quota_key_)
, cluster(cluster_)
, cluster_secret(cluster_secret_)
@ -141,7 +143,7 @@ void Connection::connect(const ConnectionTimeouts & timeouts)
async_callback(socket->impl()->sockfd(), connection_timeout, AsyncEventTimeoutType::CONNECT, description, AsyncTaskExecutor::READ | AsyncTaskExecutor::WRITE | AsyncTaskExecutor::ERROR);
if (auto err = socket->impl()->socketError())
socket->impl()->error(err); // Throws an exception
socket->impl()->error(err); // Throws an exception /// NOLINT(readability-static-accessed-through-instance)
socket->setBlocking(true);
}
@ -276,17 +278,6 @@ void Connection::disconnect()
}
String Connection::packStringForSshSign(String challenge)
{
String message;
message.append(std::to_string(DBMS_TCP_PROTOCOL_VERSION));
message.append(default_database);
message.append(user);
message.append(challenge);
return message;
}
void Connection::sendHello()
{
/** Disallow control characters in user controlled parameters
@ -334,10 +325,10 @@ void Connection::sendHello()
#endif
}
#if USE_SSH
/// Just inform server that we will authenticate using SSH keys.
else if (!ssh_private_key.isEmpty())
{
writeStringBinary(fmt::format("{}{}", EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER, user), *out);
/// Inform server that we will authenticate using SSH keys.
writeStringBinary(String(EncodedUserInfo::SSH_KEY_AUTHENTICAION_MARKER) + user, *out);
writeStringBinary(password, *out);
performHandshakeForSSHAuth();
@ -361,9 +352,9 @@ void Connection::sendAddendum()
}
#if USE_SSH
void Connection::performHandshakeForSSHAuth()
{
#if USE_SSH
String challenge;
{
writeVarUInt(Protocol::Client::SSHChallengeRequest, *out);
@ -388,12 +379,23 @@ void Connection::performHandshakeForSSHAuth()
}
writeVarUInt(Protocol::Client::SSHChallengeResponse, *out);
String to_sign = packStringForSshSign(challenge);
auto pack_string_for_ssh_sign = [&](String challenge_)
{
String message;
message.append(std::to_string(DBMS_TCP_PROTOCOL_VERSION));
message.append(default_database);
message.append(user);
message.append(challenge_);
return message;
};
String to_sign = pack_string_for_ssh_sign(challenge);
String signature = ssh_private_key.signString(to_sign);
writeStringBinary(signature, *out);
out->next();
#endif
}
#endif
void Connection::receiveHello(const Poco::Timespan & handshake_timeout)

View File

@ -1,10 +1,9 @@
#pragma once
#include <Poco/Net/StreamSocket.h>
#include <Common/SSH/Wrappers.h>
#include <Common/callOnce.h>
#include <Common/SSHWrapper.h>
#include <Client/IServerConnection.h>
#include <Core/Defines.h>
@ -53,7 +52,7 @@ public:
Connection(const String & host_, UInt16 port_,
const String & default_database_,
const String & user_, const String & password_,
const ssh::SSHKey & ssh_private_key_,
const SSHKey & ssh_private_key_,
const String & quota_key_,
const String & cluster_,
const String & cluster_secret_,
@ -170,7 +169,9 @@ private:
String default_database;
String user;
String password;
ssh::SSHKey ssh_private_key;
#if USE_SSH
SSHKey ssh_private_key;
#endif
String quota_key;
/// For inter-server authorization
@ -265,9 +266,10 @@ private:
void connect(const ConnectionTimeouts & timeouts);
void sendHello();
String packStringForSshSign(String challenge);
#if USE_SSH
void performHandshakeForSSHAuth();
#endif
void sendAddendum();
void receiveHello(const Poco::Timespan & handshake_timeout);

View File

@ -1,11 +1,10 @@
#include "ConnectionParameters.h"
#include <fstream>
#include <Core/Defines.h>
#include <Core/Protocol.h>
#include <Core/Types.h>
#include <IO/ConnectionTimeouts.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/SSH/Wrappers.h>
#include <Common/Exception.h>
#include <Common/isLocalAddress.h>
#include <Common/DNSResolver.h>
@ -88,19 +87,19 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati
}
else
{
std::string prompt{"Enter your private key passphrase (leave empty for no passphrase): "};
std::string prompt{"Enter your SSH private key passphrase (leave empty for no passphrase): "};
char buf[1000] = {};
if (auto * result = readpassphrase(prompt.c_str(), buf, sizeof(buf), 0))
passphrase = result;
}
ssh::SSHKey key = ssh::SSHKeyFactory::makePrivateFromFile(filename, passphrase);
SSHKey key = SSHKeyFactory::makePrivateKeyFromFile(filename, passphrase);
if (!key.isPrivate())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Found public key in file: {} but expected private", filename);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "File {} did not contain a private key (is it a public key?)", filename);
ssh_private_key = std::move(key);
#else
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without OpenSSL");
throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "SSH is disabled, because ClickHouse is built without libssh");
#endif
}

View File

@ -1,9 +1,10 @@
#pragma once
#include <string>
#include <Common/SSHWrapper.h>
#include <Core/Protocol.h>
#include <IO/ConnectionTimeouts.h>
#include <Common/SSH/Wrappers.h>
#include <string>
namespace Poco::Util
{
@ -20,7 +21,7 @@ struct ConnectionParameters
std::string user;
std::string password;
std::string quota_key;
ssh::SSHKey ssh_private_key;
SSHKey ssh_private_key;
Protocol::Secure security = Protocol::Secure::Disable;
Protocol::Compression compression = Protocol::Compression::Enable;
ConnectionTimeouts timeouts;

View File

@ -123,7 +123,7 @@ protected:
{
return std::make_shared<Connection>(
host, port,
default_database, user, password, ssh::SSHKey(), quota_key,
default_database, user, password, SSHKey(), quota_key,
cluster, cluster_secret,
client_name, compression, secure);
}

View File

@ -940,7 +940,7 @@ void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo &
if (nested_node)
{
/// Find any leaf of Nested subcolumn.
const auto * leaf = subcolumns.findLeaf(nested_node, [&](const auto &) { return true; });
const auto * leaf = Subcolumns::findLeaf(nested_node, [&](const auto &) { return true; });
assert(leaf);
/// Recreate subcolumn with default values and the same sizes of arrays.
@ -983,7 +983,7 @@ const ColumnObject::Subcolumns::Node * ColumnObject::getLeafOfTheSameNested(cons
while (current_node)
{
/// Try to find the first Nested up to the current node.
const auto * node_nested = subcolumns.findParent(current_node,
const auto * node_nested = Subcolumns::findParent(current_node,
[](const auto & candidate) { return candidate.isNested(); });
if (!node_nested)
@ -993,7 +993,7 @@ const ColumnObject::Subcolumns::Node * ColumnObject::getLeafOfTheSameNested(cons
/// for the last rows.
/// If there are no leaves, skip current node and find
/// the next node up to the current.
leaf = subcolumns.findLeaf(node_nested,
leaf = Subcolumns::findLeaf(node_nested,
[&](const auto & candidate)
{
return candidate.data.size() > old_size;

View File

@ -16,6 +16,7 @@
#include <Poco/Net/HTTPRequest.h>
#include <Poco/Net/HTTPResponse.h>
#include <Poco/Net/HTTPStream.h>
#include <Poco/Net/NetException.h>
#include <Poco/Timespan.h>
#include <queue>
@ -83,17 +84,15 @@ namespace
}
size_t roundUp(size_t x, size_t rounding)
constexpr size_t roundUp(size_t x, size_t rounding)
{
chassert(rounding > 0);
return (x + (rounding - 1)) / rounding * rounding;
}
Poco::Timespan divide(const Poco::Timespan span, int divisor)
{
return Poco::Timespan(Poco::Timestamp::TimeDiff(span.totalMicroseconds() / divisor));
return (x + rounding) / rounding * rounding;
}
static_assert(roundUp(10000, 100) == 10100);
static_assert(roundUp(10001, 100) == 10100);
static_assert(roundUp(10099, 100) == 10100);
static_assert(roundUp(10100, 100) == 10200);
}
namespace DB
@ -202,8 +201,9 @@ public:
if (total_connections_in_group >= limits.warning_limit && total_connections_in_group >= mute_warning_until)
{
LOG_WARNING(log, "Too many active sessions in group {}, count {}, warning limit {}", type, total_connections_in_group, limits.warning_limit);
mute_warning_until = roundUp(total_connections_in_group, HTTPConnectionPools::Limits::warning_step);
LOG_WARNING(log, "Too many active sessions in group {}, count {}, warning limit {}, next warning at {}",
type, total_connections_in_group, limits.warning_limit, mute_warning_until);
}
}
@ -213,7 +213,8 @@ public:
--total_connections_in_group;
const size_t reduced_warning_limit = limits.warning_limit > 10 ? limits.warning_limit - 10 : 1;
const size_t gap = 20;
const size_t reduced_warning_limit = limits.warning_limit > gap ? limits.warning_limit - gap : 1;
if (mute_warning_until > 0 && total_connections_in_group < reduced_warning_limit)
{
LOG_WARNING(log, "Sessions count is OK in the group {}, count {}", type, total_connections_in_group);
@ -273,9 +274,15 @@ private:
public:
using Ptr = std::shared_ptr<PooledConnection>;
using Session::mustReconnect;
void markAsExpired()
{
isExpired = true;
}
void reconnect() override
{
ProfileEvents::increment(metrics.reset);
Session::close();
if (auto lock = pool.lock())
@ -283,6 +290,7 @@ private:
auto timeouts = getTimeouts(*this);
auto new_connection = lock->getConnection(timeouts);
Session::assign(*new_connection);
Session::setKeepAliveRequest(Session::getKeepAliveRequest() + 1);
}
else
{
@ -304,6 +312,12 @@ private:
Session::getPort());
}
Poco::Timespan idleTime()
{
Poco::Timestamp now;
return now - Session::getLastRequest();
}
void flushRequest() override
{
if (bool(request_stream))
@ -335,6 +349,7 @@ private:
std::ostream & sendRequest(Poco::Net::HTTPRequest & request) override
{
auto idle = idleTime();
std::ostream & result = Session::sendRequest(request);
result.exceptions(std::ios::badbit);
@ -392,10 +407,11 @@ private:
}
response_stream = nullptr;
if (auto lock = pool.lock())
lock->atConnectionDestroy(*this);
else
ProfileEvents::increment(metrics.reset);
group->atConnectionDestroy();
if (!isExpired)
if (auto lock = pool.lock())
lock->atConnectionDestroy(*this);
CurrentMetrics::sub(metrics.active_count);
}
@ -404,10 +420,18 @@ private:
friend class EndpointConnectionPool;
template <class... Args>
explicit PooledConnection(EndpointConnectionPool::WeakPtr pool_, IHTTPConnectionPoolForEndpoint::Metrics metrics_, Args &&... args)
: Session(args...), pool(std::move(pool_)), metrics(std::move(metrics_))
explicit PooledConnection(
EndpointConnectionPool::WeakPtr pool_,
ConnectionGroup::Ptr group_,
IHTTPConnectionPoolForEndpoint::Metrics metrics_,
Args &&... args)
: Session(std::forward<Args>(args)...)
, pool(std::move(pool_))
, group(group_)
, metrics(std::move(metrics_))
{
CurrentMetrics::add(metrics.active_count);
group->atConnectionCreate();
}
template <class... Args>
@ -433,10 +457,12 @@ private:
return request_stream_completed && response_stream_completed;
}
WeakPtr pool;
EndpointConnectionPool::WeakPtr pool;
ConnectionGroup::Ptr group;
IHTTPConnectionPoolForEndpoint::Metrics metrics;
bool isExpired = false;
Poco::Logger * log = &Poco::Logger::get("PooledConnection");
LoggerPtr log = getLogger("PooledConnection");
std::ostream * request_stream = nullptr;
std::istream * response_stream = nullptr;
@ -484,7 +510,6 @@ public:
IHTTPConnectionPoolForEndpoint::ConnectionPtr getConnection(const ConnectionTimeouts & timeouts) override
{
Poco::Timestamp now;
std::vector<ConnectionPtr> expired_connections;
SCOPE_EXIT({
@ -494,8 +519,9 @@ public:
{
std::lock_guard lock(mutex);
expired_connections.reserve(stored_connections.size());
wipeExpiredImpl(expired_connections, now);
wipeExpiredImpl(expired_connections);
if (!stored_connections.empty())
{
@ -526,7 +552,6 @@ public:
size_t wipeExpired() override
{
Poco::Timestamp now;
std::vector<ConnectionPtr> expired_connections;
SCOPE_EXIT({
@ -535,25 +560,29 @@ public:
});
std::lock_guard lock(mutex);
return wipeExpiredImpl(expired_connections, now);
return wipeExpiredImpl(expired_connections);
}
size_t wipeExpiredImpl(std::vector<ConnectionPtr> & expired_connections, Poco::Timestamp now) TSA_REQUIRES(mutex)
size_t wipeExpiredImpl(std::vector<ConnectionPtr> & expired_connections) TSA_REQUIRES(mutex)
{
SCOPE_EXIT({
CurrentMetrics::sub(getMetrics().stored_count, expired_connections.size());
ProfileEvents::increment(getMetrics().expired, expired_connections.size());
});
auto isSoftLimitReached = group->isSoftLimitReached();
while (!stored_connections.empty())
{
auto connection = stored_connections.top();
if (!isExpired(now, connection))
if (!isExpired(connection, isSoftLimitReached))
return stored_connections.size();
stored_connections.pop();
connection->markAsExpired();
expired_connections.push_back(connection);
}
CurrentMetrics::sub(getMetrics().stored_count, expired_connections.size());
ProfileEvents::increment(getMetrics().expired, expired_connections.size());
return stored_connections.size();
}
@ -569,57 +598,53 @@ private:
WeakPtr getWeakFromThis() { return EndpointConnectionPool::weak_from_this(); }
bool isExpired(Poco::Timestamp & now, ConnectionPtr connection)
bool isExpired(ConnectionPtr connection, bool isSoftLimitReached) TSA_REQUIRES(mutex)
{
if (group->isSoftLimitReached())
return now > (connection->getLastRequest() + divide(connection->getKeepAliveTimeout(), 10));
return now > connection->getLastRequest() + connection->getKeepAliveTimeout();
if (isSoftLimitReached)
return connection->isKeepAliveExpired(0.1);
return connection->isKeepAliveExpired(0.8);
}
ConnectionPtr allocateNewConnection()
ConnectionPtr prepareNewConnection(const ConnectionTimeouts & timeouts)
{
ConnectionPtr connection = PooledConnection::create(this->getWeakFromThis(), getMetrics(), host, port);
auto connection = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port);
connection->setKeepAlive(true);
setTimeouts(*connection, timeouts);
if (!proxy_configuration.isEmpty())
{
connection->setProxyConfig(proxyConfigurationToPocoProxyConfig(proxy_configuration));
}
group->atConnectionCreate();
return connection;
}
ConnectionPtr prepareNewConnection(const ConnectionTimeouts & timeouts)
{
auto address = HostResolversPool::instance().getResolver(host)->resolve();
auto session = allocateNewConnection();
setTimeouts(*session, timeouts);
session->setResolvedHost(*address);
connection->setResolvedHost(*address);
try
{
auto timer = CurrentThread::getProfileEvents().timer(getMetrics().elapsed_microseconds);
session->doConnect();
connection->doConnect();
}
catch (...)
{
address.setFail();
ProfileEvents::increment(getMetrics().errors);
session->reset();
connection->reset();
throw;
}
ProfileEvents::increment(getMetrics().created);
return session;
return connection;
}
void atConnectionDestroy(PooledConnection & connection)
{
group->atConnectionDestroy();
if (connection.getKeepAliveRequest() >= connection.getKeepAliveMaxRequests())
{
ProfileEvents::increment(getMetrics().expired, 1);
return;
}
if (!connection.connected() || connection.mustReconnect() || !connection.isCompleted() || connection.buffered()
|| group->isStoreLimitReached())
@ -628,17 +653,17 @@ private:
return;
}
auto connection_to_store = allocateNewConnection();
auto connection_to_store = PooledConnection::create(this->getWeakFromThis(), group, getMetrics(), host, port);
connection_to_store->assign(connection);
CurrentMetrics::add(getMetrics().stored_count, 1);
ProfileEvents::increment(getMetrics().preserved, 1);
{
MemoryTrackerSwitcher switcher{&total_memory_tracker};
std::lock_guard lock(mutex);
stored_connections.push(connection_to_store);
}
CurrentMetrics::add(getMetrics().stored_count, 1);
ProfileEvents::increment(getMetrics().preserved, 1);
}
@ -726,14 +751,13 @@ createConnectionPool(ConnectionGroup::Ptr group, std::string host, UInt16 port,
class HTTPConnectionPools::Impl
{
private:
const size_t DEFAULT_WIPE_TIMEOUT_SECONDS = 5 * 60;
const size_t DEFAULT_WIPE_TIMEOUT_SECONDS = 10 * 60;
const Poco::Timespan wipe_timeout = Poco::Timespan(DEFAULT_WIPE_TIMEOUT_SECONDS, 0);
ConnectionGroup::Ptr disk_group = std::make_shared<ConnectionGroup>(HTTPConnectionGroupType::DISK);
ConnectionGroup::Ptr storage_group = std::make_shared<ConnectionGroup>(HTTPConnectionGroupType::STORAGE);
ConnectionGroup::Ptr http_group = std::make_shared<ConnectionGroup>(HTTPConnectionGroupType::HTTP);
/// If multiple mutexes are held simultaneously,
/// they should be locked in this order:
/// HTTPConnectionPools::mutex, then EndpointConnectionPool::mutex, then ConnectionGroup::mutex.

View File

@ -1,4 +1,5 @@
#include <Common/SSH/Wrappers.h>
#include <Common/SSHWrapper.h>
# if USE_SSH
# include <stdexcept>
@ -10,6 +11,14 @@
# pragma clang diagnostic pop
namespace DB
{
namespace ErrorCodes
{
extern const int LIBSSH_ERROR;
}
namespace
{
@ -18,17 +27,19 @@ class SSHString
public:
explicit SSHString(std::string_view input)
{
string = ssh_string_new(input.size());
ssh_string_fill(string, input.data(), input.size());
if (string = ssh_string_new(input.size()); string == nullptr)
throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't create SSHString");
if (int rc = ssh_string_fill(string, input.data(), input.size()); rc != SSH_OK)
throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't create SSHString");
}
explicit SSHString(ssh_string c_other) { string = c_other; }
explicit SSHString(ssh_string other) { string = other; }
ssh_string get() { return string; }
String toString()
{
return String(ssh_string_get_char(string), ssh_string_len(string));
return {ssh_string_get_char(string), ssh_string_len(string)};
}
~SSHString()
@ -42,46 +53,28 @@ private:
}
namespace DB
{
namespace ErrorCodes
{
extern const int LIBSSH_ERROR;
}
namespace ssh
{
SSHKey SSHKeyFactory::makePrivateFromFile(String filename, String passphrase)
SSHKey SSHKeyFactory::makePrivateKeyFromFile(String filename, String passphrase)
{
ssh_key key;
int rc = ssh_pki_import_privkey_file(filename.c_str(), passphrase.c_str(), nullptr, nullptr, &key);
if (rc != SSH_OK)
{
if (int rc = ssh_pki_import_privkey_file(filename.c_str(), passphrase.c_str(), nullptr, nullptr, &key); rc != SSH_OK)
throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't import SSH private key from file");
}
return SSHKey(key);
}
SSHKey SSHKeyFactory::makePublicFromFile(String filename)
SSHKey SSHKeyFactory::makePublicKeyFromFile(String filename)
{
ssh_key key;
int rc = ssh_pki_import_pubkey_file(filename.c_str(), &key);
if (rc != SSH_OK)
if (int rc = ssh_pki_import_pubkey_file(filename.c_str(), &key); rc != SSH_OK)
throw Exception(ErrorCodes::LIBSSH_ERROR, "Can't import SSH public key from file");
return SSHKey(key);
}
SSHKey SSHKeyFactory::makePublicFromBase64(String base64_key, String type_name)
SSHKey SSHKeyFactory::makePublicKeyFromBase64(String base64_key, String type_name)
{
ssh_key key;
auto key_type = ssh_key_type_from_name(type_name.c_str());
int rc = ssh_pki_import_pubkey_base64(base64_key.c_str(), key_type, &key);
if (rc != SSH_OK)
if (int rc = ssh_pki_import_pubkey_base64(base64_key.c_str(), key_type, &key); rc != SSH_OK)
throw Exception(ErrorCodes::LIBSSH_ERROR, "Bad SSH public key provided");
return SSHKey(key);
}
@ -90,6 +83,12 @@ SSHKey::SSHKey(const SSHKey & other)
key = ssh_key_dup(other.key);
}
SSHKey::SSHKey(SSHKey && other) noexcept
{
key = other.key;
other.key = nullptr;
}
SSHKey & SSHKey::operator=(const SSHKey & other)
{
ssh_key_free(key);
@ -119,13 +118,11 @@ bool SSHKey::isEqual(const SSHKey & other) const
String SSHKey::signString(std::string_view input) const
{
SSHString input_str(input);
ssh_string c_output = nullptr;
int rc = pki_sign_string(key, input_str.get(), &c_output);
if (rc != SSH_OK)
ssh_string output = nullptr;
if (int rc = pki_sign_string(key, input_str.get(), &output); rc != SSH_OK)
throw Exception(ErrorCodes::LIBSSH_ERROR, "Error singing with ssh key");
SSHString output(c_output);
return output.toString();
SSHString output_str(output);
return output_str.toString();
}
bool SSHKey::verifySignature(std::string_view signature, std::string_view original) const
@ -149,18 +146,15 @@ namespace
{
struct CStringDeleter
{
[[maybe_unused]] void operator()(char * ptr) const { std::free(ptr); }
void operator()(char * ptr) const { std::free(ptr); }
};
}
String SSHKey::getBase64() const
{
char * buf = nullptr;
int rc = ssh_pki_export_pubkey_base64(key, &buf);
if (rc != SSH_OK)
if (int rc = ssh_pki_export_pubkey_base64(key, &buf); rc != SSH_OK)
throw DB::Exception(DB::ErrorCodes::LIBSSH_ERROR, "Failed to export public key to base64");
/// Create a String from cstring, which makes a copy of the first one and requires freeing memory after it
/// This is to safely manage buf memory
std::unique_ptr<char, CStringDeleter> buf_ptr(buf);
@ -177,7 +171,6 @@ SSHKey::~SSHKey()
ssh_key_free(key); // it's safe free from libssh
}
}
}
#endif

View File

@ -1,20 +1,18 @@
#pragma once
#include <Common/Exception.h>
#include "config.h"
#if USE_SSH
# include <string_view>
# include <base/types.h>
#include <Common/Exception.h>
#include <string_view>
#include <base/types.h>
#include "config.h"
#if USE_SSH
using ssh_key = struct ssh_key_struct *;
namespace DB
{
namespace ssh
{
class SSHKeyFactory;
class SSHKey
{
public:
@ -22,11 +20,7 @@ public:
~SSHKey();
SSHKey(const SSHKey & other);
SSHKey(SSHKey && other) noexcept
{
key = other.key;
other.key = nullptr;
}
SSHKey(SSHKey && other) noexcept;
SSHKey & operator=(const SSHKey & other);
SSHKey & operator=(SSHKey && other) noexcept;
@ -43,7 +37,7 @@ public:
String getBase64() const;
String getKeyType() const;
friend SSHKeyFactory;
friend class SSHKeyFactory;
private:
explicit SSHKey(ssh_key key_) : key(key_) { }
ssh_key key = nullptr;
@ -56,17 +50,14 @@ public:
/// The check whether the path is allowed to read for ClickHouse has
/// (e.g. a file is inside `user_files` directory)
/// to be done outside of this functions.
static SSHKey makePrivateFromFile(String filename, String passphrase);
static SSHKey makePublicFromFile(String filename);
static SSHKey makePublicFromBase64(String base64_key, String type_name);
static SSHKey makePrivateKeyFromFile(String filename, String passphrase);
static SSHKey makePublicKeyFromFile(String filename);
static SSHKey makePublicKeyFromBase64(String base64_key, String type_name);
};
}
}
#else
namespace ssh
{
class SSHKey
{
public:
@ -74,5 +65,4 @@ public:
[[ noreturn ]] bool isEmpty() { std::terminate(); }
[[ noreturn ]] String signString(std::string_view) const { std::terminate(); }
};
}
#endif

View File

@ -18,13 +18,10 @@
#include <filesystem>
#include <map>
#include <mutex>
#include <sstream>
#include <unordered_map>
#include <fmt/format.h>
#include <libunwind.h>
#include "config.h"
#include <boost/algorithm/string/split.hpp>
#if defined(OS_DARWIN)
@ -481,7 +478,17 @@ void StackTrace::toStringEveryLine(void ** frame_pointers_raw, size_t offset, si
toStringEveryLineImpl(true, {frame_pointers, offset, size}, std::move(callback));
}
using StackTraceCache = std::map<StackTraceTriple, String, std::less<>>;
struct CacheEntry
{
std::optional<String> stacktrace_string;
bool to_string_in_progress = false;
std::condition_variable cv;
};
using CacheEntryPtr = std::shared_ptr<CacheEntry>;
using StackTraceCache = std::map<StackTraceTriple, CacheEntryPtr, std::less<>>;
static StackTraceCache & cacheInstance()
{
@ -493,23 +500,63 @@ static std::mutex stacktrace_cache_mutex;
String toStringCached(const StackTrace::FramePointers & pointers, size_t offset, size_t size)
{
const StackTraceRefTriple key{pointers, offset, size};
/// Calculation of stack trace text is extremely slow.
/// We use simple cache because otherwise the server could be overloaded by trash queries.
/// Note that this cache can grow unconditionally, but practically it should be small.
std::lock_guard lock{stacktrace_cache_mutex};
std::unique_lock lock{stacktrace_cache_mutex};
CacheEntryPtr cache_entry;
StackTraceCache & cache = cacheInstance();
const StackTraceRefTriple key{pointers, offset, size};
if (auto it = cache.find(key); it != cache.end())
return it->second;
{
cache_entry = it->second;
}
else
{
auto [new_it, inserted] = cache.emplace(StackTraceTriple{pointers, offset, size}, std::make_shared<CacheEntry>());
chassert(inserted);
cache_entry = new_it->second;
}
if (!cache_entry->to_string_in_progress && cache_entry->stacktrace_string.has_value())
return *cache_entry->stacktrace_string;
if (cache_entry->to_string_in_progress)
{
cache_entry->cv.wait(lock, [&]{ return !cache_entry->to_string_in_progress; });
if (cache_entry->stacktrace_string.has_value())
return *cache_entry->stacktrace_string;
}
cache_entry->to_string_in_progress = true;
lock.unlock();
String stacktrace_string;
try
{
DB::WriteBufferFromOwnString out;
toStringEveryLineImpl(false, key, [&](std::string_view str) { out << str << '\n'; });
return cache.emplace(StackTraceTriple{pointers, offset, size}, out.str()).first->second;
stacktrace_string = out.str();
}
catch (...)
{
lock.lock();
cache_entry->to_string_in_progress = false;
lock.unlock();
cache_entry->cv.notify_one();
throw;
}
lock.lock();
cache_entry->to_string_in_progress = false;
cache_entry->stacktrace_string = stacktrace_string;
lock.unlock();
cache_entry->cv.notify_all();
return stacktrace_string;
}
std::string StackTrace::toString() const

View File

@ -2,7 +2,6 @@
#include <Common/HTTPConnectionPool.h>
#include <Poco/URI.h>
#include <Poco/Net/ServerSocket.h>
#include <Poco/Net/MessageHeader.h>
#include <Poco/Net/HTTPServerRequest.h>
#include <Poco/Net/HTTPServerResponse.h>
@ -17,6 +16,40 @@
namespace
{
template<class T>
class SafeHandler
{
public:
using Ptr = std::shared_ptr<SafeHandler<T>>;
SafeHandler() = default;
SafeHandler(SafeHandler<T>&) = delete;
SafeHandler& operator=(SafeHandler<T>&) = delete;
T get()
{
std::lock_guard lock(mutex);
return obj;
}
void set(T && options_)
{
std::lock_guard lock(mutex);
obj = std::move(options_);
}
protected:
std::mutex mutex;
T obj = {};
};
struct RequestOptions
{
size_t slowdown_receive = 0;
int overwrite_keep_alive_timeout = 0;
int overwrite_keep_alive_max_requests = 10;
};
size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count = std::numeric_limits<size_t>::max())
{
const size_t buffer_size = 4096;
@ -47,13 +80,21 @@ size_t stream_copy_n(std::istream & in, std::ostream & out, std::size_t count =
class MockRequestHandler : public Poco::Net::HTTPRequestHandler
{
public:
explicit MockRequestHandler(std::shared_ptr<std::atomic<size_t>> slowdown_)
: slowdown(std::move(slowdown_))
explicit MockRequestHandler(SafeHandler<RequestOptions>::Ptr options_)
: options(options_)
{
}
void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override
{
int value = request.getKeepAliveTimeout();
ASSERT_GT(value, 0);
auto params = options->get();
if (params.overwrite_keep_alive_timeout > 0)
response.setKeepAliveTimeout(params.overwrite_keep_alive_timeout, params.overwrite_keep_alive_max_requests);
response.setStatus(Poco::Net::HTTPResponse::HTTP_OK);
auto size = request.getContentLength();
if (size > 0)
@ -61,28 +102,29 @@ public:
else
response.setChunkedTransferEncoding(true); // or chunk encoding
sleepForSeconds(*slowdown);
if (params.slowdown_receive > 0)
sleepForSeconds(params.slowdown_receive);
stream_copy_n(request.stream(), response.send(), size);
}
std::shared_ptr<std::atomic<size_t>> slowdown;
SafeHandler<RequestOptions>::Ptr options;
};
class HTTPRequestHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory
{
public:
explicit HTTPRequestHandlerFactory(std::shared_ptr<std::atomic<size_t>> slowdown_)
: slowdown(std::move(slowdown_))
explicit HTTPRequestHandlerFactory(SafeHandler<RequestOptions>::Ptr options_)
: options(options_)
{
}
Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest &) override
{
return new MockRequestHandler(slowdown);
return new MockRequestHandler(options);
}
std::shared_ptr<std::atomic<size_t>> slowdown;
SafeHandler<RequestOptions>::Ptr options;
};
}
@ -94,6 +136,8 @@ class ConnectionPoolTest : public testing::Test {
protected:
ConnectionPoolTest()
{
options = std::make_shared<SafeHandler<RequestOptions>>();
startServer();
}
@ -102,7 +146,7 @@ protected:
DB::HTTPConnectionPools::Limits def_limits{};
DB::HTTPConnectionPools::instance().setLimits(def_limits, def_limits, def_limits);
setSlowDown(0);
options->set(RequestOptions());
DB::HTTPConnectionPools::instance().dropCache();
DB::CurrentThread::getProfileEvents().reset();
@ -129,7 +173,7 @@ protected:
void startServer()
{
server_data.reset();
server_data.handler_factory = new HTTPRequestHandlerFactory(slowdown_receive);
server_data.handler_factory = new HTTPRequestHandlerFactory(options);
server_data.server = std::make_unique<Poco::Net::HTTPServer>(
server_data.handler_factory, server_data.port);
@ -143,11 +187,21 @@ protected:
void setSlowDown(size_t seconds)
{
*slowdown_receive = seconds;
auto opt = options->get();
opt.slowdown_receive = seconds;
options->set(std::move(opt));
}
void setOverWriteKeepAlive(size_t seconds, int max_requests)
{
auto opt = options->get();
opt.overwrite_keep_alive_timeout = int(seconds);
opt.overwrite_keep_alive_max_requests= max_requests;
options->set(std::move(opt));
}
DB::ConnectionTimeouts timeouts;
std::shared_ptr<std::atomic<size_t>> slowdown_receive = std::make_shared<std::atomic<size_t>>(0);
SafeHandler<RequestOptions>::Ptr options;
struct ServerData
{
@ -182,7 +236,7 @@ protected:
void wait_until(std::function<bool()> pred)
{
while (!pred())
sleepForMilliseconds(250);
sleepForMilliseconds(10);
}
void echoRequest(String data, HTTPSession & session)
@ -245,45 +299,52 @@ TEST_F(ConnectionPoolTest, CanRequest)
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
auto metrics = pool->getMetrics();
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, CanPreserve)
{
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
}
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
wait_until([&] () { return getServer().currentConnections() == 1; });
ASSERT_EQ(1, getServer().currentConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
}
TEST_F(ConnectionPoolTest, CanReuse)
{
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
// DB::setReuseTag(*connection);
}
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().stored_count));
{
auto connection = pool->getConnection(timeouts);
ASSERT_EQ(1, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
wait_until([&] () { return getServer().currentConnections() == 1; });
ASSERT_EQ(1, getServer().currentConnections());
@ -293,6 +354,11 @@ TEST_F(ConnectionPoolTest, CanReuse)
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, getServer().currentConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
connection->reset();
}
@ -303,15 +369,16 @@ TEST_F(ConnectionPoolTest, CanReuse)
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
}
TEST_F(ConnectionPoolTest, CanReuse10)
{
auto pool = getPool();
auto metrics = pool->getMetrics();
for (int i = 0; i < 10; ++i)
{
@ -328,16 +395,23 @@ TEST_F(ConnectionPoolTest, CanReuse10)
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(1, getServer().totalConnections());
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, CanReuse5)
{
timeouts.withHTTPKeepAliveTimeout(1);
auto ka = Poco::Timespan(1, 0); // 1 seconds
timeouts.withHTTPKeepAliveTimeout(ka);
auto pool = getPool();
auto metrics = pool->getMetrics();
std::vector<DB::HTTPSessionPtr> connections;
connections.reserve(5);
@ -347,11 +421,14 @@ TEST_F(ConnectionPoolTest, CanReuse5)
}
connections.clear();
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(5, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(5, CurrentMetrics::get(metrics.stored_count));
wait_until([&] () { return getServer().currentConnections() == 5; });
ASSERT_EQ(5, getServer().currentConnections());
@ -363,35 +440,56 @@ TEST_F(ConnectionPoolTest, CanReuse5)
echoRequest("Hello", *connection);
}
ASSERT_EQ(5, getServer().totalConnections());
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(5, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(5, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(5, CurrentMetrics::get(metrics.stored_count));
/// wait until all connections are timeouted
wait_until([&] () { return getServer().currentConnections() == 0; });
{
// just to trigger pool->wipeExpired();
auto connection = pool->getConnection(timeouts);
connection->reset();
}
ASSERT_EQ(6, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(5, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, CanReconnectAndCreate)
{
auto pool = getPool();
auto metrics = pool->getMetrics();
std::vector<HTTPSessionPtr> in_use;
const size_t count = 2;
const size_t count = 3;
for (int i = 0; i < count; ++i)
{
auto connection = pool->getConnection(timeouts);
// DB::setReuseTag(*connection);
in_use.push_back(connection);
}
ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(count, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(count, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
auto connection = std::move(in_use.back());
in_use.pop_back();
@ -402,28 +500,39 @@ TEST_F(ConnectionPoolTest, CanReconnectAndCreate)
echoRequest("Hello", *connection);
connection->reset();
ASSERT_EQ(count+1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
wait_until([&] () { return getServer().currentConnections() == 1; });
ASSERT_EQ(1, getServer().currentConnections());
ASSERT_EQ(count+1, getServer().totalConnections());
ASSERT_EQ(count+1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(count, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, CanReconnectAndReuse)
{
auto ka = Poco::Timespan(1, 0); // 1 seconds
timeouts.withHTTPKeepAliveTimeout(ka);
auto pool = getPool();
auto metrics = pool->getMetrics();
std::vector<HTTPSessionPtr> in_use;
const size_t count = 2;
const size_t count = 3;
for (int i = 0; i < count; ++i)
{
auto connection = pool->getConnection(timeouts);
/// make some request in order to show to the server the keep alive headers
echoRequest("Hello", *connection);
in_use.push_back(std::move(connection));
}
in_use.clear();
for (int i = 0; i < count; ++i)
{
auto connection = pool->getConnection(timeouts);
// DB::setReuseTag(*connection);
in_use.push_back(std::move(connection));
}
@ -441,11 +550,16 @@ TEST_F(ConnectionPoolTest, CanReconnectAndReuse)
wait_until([&] () { return getServer().currentConnections() == 0; });
ASSERT_EQ(0, getServer().currentConnections());
ASSERT_EQ(2, getServer().totalConnections());
ASSERT_EQ(count, getServer().totalConnections());
ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(count, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(count + count - 1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(count + 1, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(count-1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(count-2, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, ReceiveTimeout)
@ -454,6 +568,7 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout)
timeouts.withReceiveTimeout(1);
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
@ -462,10 +577,14 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout)
);
}
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
{
timeouts.withReceiveTimeout(3);
@ -475,10 +594,14 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout)
);
}
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
{
/// timeouts have effect for reused session
@ -489,10 +612,14 @@ TEST_F(ConnectionPoolTest, ReceiveTimeout)
);
}
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reused]);
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP)
@ -500,6 +627,7 @@ TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP)
std::string_view message = "Hello ReadWriteBufferFromHTTP";
auto uri = Poco::URI(getServerUrl());
auto metrics = DB::HTTPConnectionPools::instance().getPool(DB::HTTPConnectionGroupType::HTTP, uri, DB::ProxyConfiguration{})->getMetrics();
Poco::Net::HTTPBasicCredentials empty_creds;
auto buf_from_http = DB::BuilderRWBufferFromHTTP(uri)
.withConnectionGroup(DB::HTTPConnectionGroupType::HTTP)
@ -527,6 +655,7 @@ TEST_F(ConnectionPoolTest, ReadWriteBufferFromHTTP)
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
@ -538,23 +667,26 @@ TEST_F(ConnectionPoolTest, HardLimit)
DB::HTTPConnectionPools::instance().setLimits(zero_limits, zero_limits, zero_limits);
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
}
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, NoReceiveCall)
{
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
@ -570,11 +702,209 @@ TEST_F(ConnectionPoolTest, NoReceiveCall)
connection->flushRequest();
}
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().active_count));
ASSERT_EQ(0, CurrentMetrics::get(pool->getMetrics().stored_count));
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().created]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[pool->getMetrics().preserved]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[pool->getMetrics().reset]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, ReconnectedWhenConnectionIsHoldTooLong)
{
auto ka = Poco::Timespan(1, 0); // 1 seconds
timeouts.withHTTPKeepAliveTimeout(ka);
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
auto fake_ka = Poco::Timespan(30 * 1000 * 1000); // 30 seconds
timeouts.withHTTPKeepAliveTimeout(fake_ka);
DB::setTimeouts(*connection, timeouts); // new keep alive timeout has no effect
wait_until([&] () { return getServer().currentConnections() == 0; });
ASSERT_EQ(1, connection->connected());
ASSERT_EQ(1, connection->getKeepAlive());
ASSERT_EQ(1000, connection->getKeepAliveTimeout().totalMilliseconds());
ASSERT_EQ(1, connection->isKeepAliveExpired(connection->getKeepAliveReliability()));
echoRequest("Hello", *connection);
}
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, ReconnectedWhenConnectionIsNearlyExpired)
{
auto ka = Poco::Timespan(1, 0); // 1 seconds
timeouts.withHTTPKeepAliveTimeout(ka);
auto pool = getPool();
auto metrics = pool->getMetrics();
{
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
}
sleepForMilliseconds(900);
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
}
}
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, ServerOverwriteKeepAlive)
{
auto ka = Poco::Timespan(30, 0); // 30 seconds
timeouts.withHTTPKeepAliveTimeout(ka);
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds());
ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds());
}
{
setOverWriteKeepAlive(1, 10);
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds());
ASSERT_EQ(1, connection->getKeepAliveTimeout().totalSeconds());
}
{
// server do not overwrite it in the following requests but client has to remember last agreed value
setOverWriteKeepAlive(0, 0);
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(30, timeouts.http_keep_alive_timeout.totalSeconds());
ASSERT_EQ(1, connection->getKeepAliveTimeout().totalSeconds());
}
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(3, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(2, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, MaxRequests)
{
auto ka = Poco::Timespan(30, 0); // 30 seconds
timeouts.withHTTPKeepAliveTimeout(ka);
auto max_requests = 5;
timeouts.http_keep_alive_max_requests = max_requests;
auto pool = getPool();
auto metrics = pool->getMetrics();
for (int i = 1; i <= max_requests - 1; ++i)
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds());
ASSERT_EQ(max_requests, connection->getKeepAliveMaxRequests());
ASSERT_EQ(i, connection->getKeepAliveRequest());
}
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(max_requests-1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(max_requests-2, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(1, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(1, CurrentMetrics::get(metrics.stored_count));
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds());
ASSERT_EQ(max_requests, connection->getKeepAliveMaxRequests());
ASSERT_EQ(max_requests, connection->getKeepAliveRequest());
}
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(max_requests-1, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(max_requests-1, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(1, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}
TEST_F(ConnectionPoolTest, ServerOverwriteMaxRequests)
{
auto ka = Poco::Timespan(30, 0); // 30 seconds
timeouts.withHTTPKeepAliveTimeout(ka);
auto pool = getPool();
auto metrics = pool->getMetrics();
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(30, connection->getKeepAliveTimeout().totalSeconds());
ASSERT_EQ(1000, connection->getKeepAliveMaxRequests());
ASSERT_EQ(1, connection->getKeepAliveRequest());
}
auto max_requests = 3;
setOverWriteKeepAlive(5, max_requests);
for (int i = 2; i <= 10*max_requests; ++i)
{
auto connection = pool->getConnection(timeouts);
echoRequest("Hello", *connection);
ASSERT_EQ(5, connection->getKeepAliveTimeout().totalSeconds());
ASSERT_EQ(max_requests, connection->getKeepAliveMaxRequests());
ASSERT_EQ(((i-1) % max_requests) + 1, connection->getKeepAliveRequest());
}
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.created]);
ASSERT_EQ(10*max_requests-10, DB::CurrentThread::getProfileEvents()[metrics.preserved]);
ASSERT_EQ(10*max_requests-10, DB::CurrentThread::getProfileEvents()[metrics.reused]);
ASSERT_EQ(0, DB::CurrentThread::getProfileEvents()[metrics.reset]);
ASSERT_EQ(10, DB::CurrentThread::getProfileEvents()[metrics.expired]);
ASSERT_EQ(0, CurrentMetrics::get(metrics.active_count));
ASSERT_EQ(0, CurrentMetrics::get(metrics.stored_count));
}

View File

@ -483,7 +483,7 @@ void testTranscoding(Timer & timer, ICompressionCodec & codec, const CodecTestSe
ASSERT_TRUE(EqualByteContainers(test_sequence.data_type->getSizeOfValueInMemory(), source_data, decoded));
const auto header_size = codec.getHeaderSize();
const auto header_size = ICompressionCodec::getHeaderSize();
const auto compression_ratio = (encoded_size - header_size) / (source_data.size() * 1.0);
if (expected_compression_ratio)

View File

@ -54,6 +54,7 @@ static constexpr auto DEFAULT_COUNT_OF_HTTP_CONNECTIONS_PER_ENDPOINT = 15;
static constexpr auto DEFAULT_TCP_KEEP_ALIVE_TIMEOUT = 290;
static constexpr auto DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT = 30;
static constexpr auto DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST = 1000;
static constexpr auto DBMS_DEFAULT_PATH = "/var/lib/clickhouse/";

View File

@ -56,10 +56,11 @@ namespace DB
namespace EncodedUserInfo
{
/// Marker of the inter-server secret (passed in the user name)
/// Marker for the inter-server secret (passed as the user name)
/// (anyway user cannot be started with a whitespace)
const char USER_INTERSERVER_MARKER[] = " INTERSERVER SECRET ";
/// Marker of the SSH keys based authentication (passed in the user name)
/// Marker for SSH-keys-based authentication (passed as the user name)
const char SSH_KEY_AUTHENTICAION_MARKER[] = " SSH KEY AUTHENTICATION ";
};
@ -160,8 +161,8 @@ namespace Protocol
ReadTaskResponse = 9, /// A filename to read from s3 (used in s3Cluster)
MergeTreeReadTaskResponse = 10, /// Coordinator's decision with a modified set of mark ranges allowed to read
SSHChallengeRequest = 11, /// Request for SSH signature challenge
SSHChallengeResponse = 12, /// Request for SSH signature challenge
SSHChallengeRequest = 11, /// Request SSH signature challenge
SSHChallengeResponse = 12, /// Reply to SSH signature challenge
MAX = SSHChallengeResponse,
};

View File

@ -128,9 +128,9 @@ namespace DB
M(Bool, format_alter_operations_with_parentheses, false, "If enabled, each operation in alter queries will be surrounded with parentheses in formatted queries to make them less ambiguous.", 0) \
M(String, default_replica_path, "/clickhouse/tables/{uuid}/{shard}", "The path to the table in ZooKeeper", 0) \
M(String, default_replica_name, "{replica}", "The replica name in ZooKeeper", 0) \
M(UInt64, disk_connections_soft_limit, 1000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \
M(UInt64, disk_connections_soft_limit, 5000, "Connections above this limit have significantly shorter time to live. The limit applies to the disks connections.", 0) \
M(UInt64, disk_connections_warn_limit, 10000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the disks connections.", 0) \
M(UInt64, disk_connections_store_limit, 12000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \
M(UInt64, disk_connections_store_limit, 30000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the disks connections.", 0) \
M(UInt64, storage_connections_soft_limit, 100, "Connections above this limit have significantly shorter time to live. The limit applies to the storages connections.", 0) \
M(UInt64, storage_connections_warn_limit, 1000, "Warning massages are written to the logs if number of in-use connections are higher than this limit. The limit applies to the storages connections.", 0) \
M(UInt64, storage_connections_store_limit, 5000, "Connections above this limit reset after use. Set to 0 to turn connection cache off. The limit applies to the storages connections.", 0) \

View File

@ -332,6 +332,7 @@ private:
const std::vector<StackTrace::FramePointers> & thread_frame_pointers,
UInt32 thread_num,
ThreadStatus * thread_ptr) const
try
{
ThreadStatus thread_status;
@ -519,7 +520,7 @@ private:
}
}
/// ClickHouse Keeper does not link to some part of Settings.
/// ClickHouse Keeper does not link to some parts of Settings.
#ifndef CLICKHOUSE_KEEPER_STANDALONE_BUILD
/// List changed settings.
if (!query_id.empty())
@ -537,12 +538,18 @@ private:
}
#endif
/// When everything is done, we will try to send these error messages to client.
/// When everything is done, we will try to send these error messages to the client.
if (thread_ptr)
thread_ptr->onFatalError();
fatal_error_printed.test_and_set();
}
catch (...)
{
/// onFault is called from the std::thread, and it should catch all exceptions; otherwise, you can get unrelated fatal errors.
PreformattedMessage message = getCurrentExceptionMessageAndPattern(true);
LOG_FATAL(getLogger(__PRETTY_FUNCTION__), message);
}
};
@ -665,7 +672,7 @@ void BaseDaemon::reloadConfiguration()
*/
config_path = config().getString("config-file", getDefaultConfigFileName());
ConfigProcessor config_processor(config_path, false, true);
config_processor.setConfigPath(fs::path(config_path).parent_path());
ConfigProcessor::setConfigPath(fs::path(config_path).parent_path());
loaded_config = config_processor.loadConfig(/* allow_zk_includes = */ true);
if (last_configuration != nullptr)

View File

@ -18,7 +18,7 @@ SerializationPtr DataTypeDate32::doGetDefaultSerialization() const
Field DataTypeDate32::getDefault() const
{
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch());
return -static_cast<Int64>(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance)
}
void registerDataTypeDate32(DataTypeFactory & factory)

View File

@ -34,7 +34,7 @@ TEST(JSONDataParser, ReadJSON)
JSONDataParser<SimdJSONParser> parser;
ReadBufferFromString buf(json_bad);
String res;
parser.readJSON(res, buf);
JSONDataParser<SimdJSONParser>::readJSON(res, buf);
ASSERT_EQ(json1, res);
}
@ -44,7 +44,7 @@ TEST(JSONDataParser, ReadJSON)
JSONDataParser<SimdJSONParser> parser;
ReadBufferFromString buf(json_bad);
String res;
parser.readJSON(res, buf);
JSONDataParser<SimdJSONParser>::readJSON(res, buf);
ASSERT_EQ(json2, res);
}
}

View File

@ -346,7 +346,7 @@ CachedOnDiskReadBufferFromFile::getReadBufferForFileSegment(FileSegment & file_s
}
auto downloader_id = file_segment.getOrSetDownloader();
if (downloader_id == file_segment.getCallerId())
if (downloader_id == FileSegment::getCallerId())
{
if (canStartFromCache(file_offset_of_buffer_end, file_segment))
{

View File

@ -76,6 +76,9 @@ std::unique_ptr<S3::Client> getClient(
client_configuration.connectTimeoutMs = config.getUInt(config_prefix + ".connect_timeout_ms", S3::DEFAULT_CONNECT_TIMEOUT_MS);
client_configuration.requestTimeoutMs = config.getUInt(config_prefix + ".request_timeout_ms", S3::DEFAULT_REQUEST_TIMEOUT_MS);
client_configuration.maxConnections = config.getUInt(config_prefix + ".max_connections", S3::DEFAULT_MAX_CONNECTIONS);
client_configuration.http_keep_alive_timeout = config.getUInt(config_prefix + ".http_keep_alive_timeout", S3::DEFAULT_KEEP_ALIVE_TIMEOUT);
client_configuration.http_keep_alive_max_requests = config.getUInt(config_prefix + ".http_keep_alive_max_requests", S3::DEFAULT_KEEP_ALIVE_MAX_REQUESTS);
client_configuration.endpointOverride = uri.endpoint;
client_configuration.s3_use_adaptive_timeouts = config.getBool(
config_prefix + ".use_adaptive_timeouts", client_configuration.s3_use_adaptive_timeouts);

View File

@ -279,7 +279,7 @@ struct ToDate32Transform32Or64Signed
static NO_SANITIZE_UNDEFINED Int32 execute(const FromType & from, const DateLUTImpl & time_zone)
{
static const Int32 daynum_min_offset = -static_cast<Int32>(time_zone.getDayNumOffsetEpoch());
static const Int32 daynum_min_offset = -static_cast<Int32>(DateLUTImpl::getDayNumOffsetEpoch());
if constexpr (date_time_overflow_behavior == FormatSettings::DateTimeOverflowBehavior::Throw)
{
@ -1092,7 +1092,7 @@ struct ConvertThroughParsing
{
if constexpr (std::is_same_v<ToDataType, DataTypeDate32>)
{
vec_to[i] = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch());
vec_to[i] = -static_cast<Int32>(DateLUT::instance().getDayNumOffsetEpoch()); /// NOLINT(readability-static-accessed-through-instance)
}
else
{

View File

@ -106,7 +106,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeRegistered(const ContextPtr & cont
if (AggregateFunctionFactory::instance().hasNameOrAlias(function_name))
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "The aggregate function '{}' already exists", function_name);
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context))
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) /// NOLINT(readability-static-accessed-through-instance)
throw Exception(ErrorCodes::FUNCTION_ALREADY_EXISTS, "User defined executable function '{}' already exists", function_name);
validateFunction(assert_cast<const ASTCreateFunctionQuery &>(create_function_query).function_core, function_name);
@ -118,7 +118,7 @@ void UserDefinedSQLFunctionFactory::checkCanBeUnregistered(const ContextPtr & co
AggregateFunctionFactory::instance().hasNameOrAlias(function_name))
throw Exception(ErrorCodes::CANNOT_DROP_FUNCTION, "Cannot drop system function '{}'", function_name);
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context))
if (UserDefinedExecutableFunctionFactory::instance().has(function_name, context)) /// NOLINT(readability-static-accessed-through-instance)
throw Exception(ErrorCodes::CANNOT_DROP_FUNCTION, "Cannot drop user defined executable function '{}'", function_name);
}

View File

@ -66,13 +66,13 @@ struct DotProduct
};
template <typename Type>
static void accumulate(State<Type> & state, Type x, Type y)
static NO_SANITIZE_UNDEFINED void accumulate(State<Type> & state, Type x, Type y)
{
state.sum += x * y;
}
template <typename Type>
static void combine(State<Type> & state, const State<Type> & other_state)
static NO_SANITIZE_UNDEFINED void combine(State<Type> & state, const State<Type> & other_state)
{
state.sum += other_state.sum;
}

View File

@ -144,7 +144,12 @@ ConnectionTimeouts ConnectionTimeouts::getAdaptiveTimeouts(const String & method
void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts)
{
session.setTimeout(timeouts.connection_timeout, timeouts.send_timeout, timeouts.receive_timeout);
session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout);
/// we can not change keep alive timeout for already initiated connections
if (!session.connected())
{
session.setKeepAliveTimeout(timeouts.http_keep_alive_timeout);
session.setKeepAliveMaxRequests(int(timeouts.http_keep_alive_max_requests));
}
}
ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session)

View File

@ -35,6 +35,8 @@ struct ConnectionTimeouts
Poco::Timespan tcp_keep_alive_timeout = Poco::Timespan(DEFAULT_TCP_KEEP_ALIVE_TIMEOUT, 0);
Poco::Timespan http_keep_alive_timeout = Poco::Timespan(DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT, 0);
size_t http_keep_alive_max_requests = DEFAULT_HTTP_KEEP_ALIVE_MAX_REQUEST;
/// Timeouts for HedgedConnections
Poco::Timespan hedged_connection_timeout = Poco::Timespan(DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, 0);
@ -69,6 +71,7 @@ APPLY_FOR_ALL_CONNECTION_TIMEOUT_MEMBERS(DECLARE_BUILDER_FOR_MEMBER)
ConnectionTimeouts & withConnectionTimeout(size_t seconds);
ConnectionTimeouts & withConnectionTimeout(Poco::Timespan span);
ConnectionTimeouts & withHTTPKeepAliveMaxRequests(size_t requests);
};
/// NOLINTBEGIN(bugprone-macro-parentheses)
@ -114,6 +117,12 @@ inline ConnectionTimeouts & ConnectionTimeouts::withConnectionTimeout(Poco::Time
return *this;
}
inline ConnectionTimeouts & ConnectionTimeouts::withHTTPKeepAliveMaxRequests(size_t requests)
{
http_keep_alive_max_requests = requests;
return *this;
}
void setTimeouts(Poco::Net::HTTPClientSession & session, const ConnectionTimeouts & timeouts);
ConnectionTimeouts getTimeouts(const Poco::Net::HTTPClientSession & session);

View File

@ -37,7 +37,7 @@ MMapReadBufferFromFileWithCache::MMapReadBufferFromFileWithCache(
MMapReadBufferFromFileWithCache::MMapReadBufferFromFileWithCache(
MMappedFileCache & cache, const std::string & file_name, size_t offset)
{
mapped = cache.getOrSet(cache.hash(file_name, offset, -1), [&]
mapped = cache.getOrSet(MMappedFileCache::hash(file_name, offset, -1), [&]
{
return std::make_shared<MMappedFile>(file_name, offset);
});

View File

@ -345,7 +345,7 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function<void()> && callable,
if (last_attempt || !is_retriable)
{
if (!mute_logging)
LOG_ERROR(log,
LOG_DEBUG(log,
"Failed to make request to '{}'{}. "
"Error: '{}'. "
"Failed at try {}/{}.",
@ -361,7 +361,7 @@ void ReadWriteBufferFromHTTP::doWithRetries(std::function<void()> && callable,
on_retry();
if (!mute_logging)
LOG_INFO(log,
LOG_TRACE(log,
"Failed to make request to '{}'{}. "
"Error: {}. "
"Failed at try {}/{}. "

View File

@ -96,9 +96,9 @@ bool isS3ExpressEndpoint(const std::string & endpoint);
struct ClientSettings
{
bool use_virtual_addressing;
bool use_virtual_addressing = false;
/// Disable checksum to avoid extra read of the input stream
bool disable_checksum;
bool disable_checksum = false;
/// Should client send ComposeObject request after upload to GCS.
///
/// Previously ComposeObject request was required to make Copy possible,
@ -108,8 +108,8 @@ struct ClientSettings
///
/// Ability to enable it preserved since likely it is required for old
/// files.
bool gcs_issue_compose_request;
bool is_s3express_bucket;
bool gcs_issue_compose_request = false;
bool is_s3express_bucket = false;
};
/// Client that improves the client from the AWS SDK

View File

@ -22,6 +22,8 @@ inline static constexpr uint64_t DEFAULT_EXPIRATION_WINDOW_SECONDS = 120;
inline static constexpr uint64_t DEFAULT_CONNECT_TIMEOUT_MS = 1000;
inline static constexpr uint64_t DEFAULT_REQUEST_TIMEOUT_MS = 30000;
inline static constexpr uint64_t DEFAULT_MAX_CONNECTIONS = 100;
inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_TIMEOUT = 5;
inline static constexpr uint64_t DEFAULT_KEEP_ALIVE_MAX_REQUESTS = 100;
/// In GCP metadata service can be accessed via DNS regardless of IPv4 or IPv6.
static inline constexpr char GCP_METADATA_SERVICE_ENDPOINT[] = "http://metadata.google.internal";

View File

@ -146,7 +146,9 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio
.withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
.withTCPKeepAliveTimeout(Poco::Timespan(
client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0));
client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))
.withHTTPKeepAliveTimeout(Poco::Timespan(client_configuration.http_keep_alive_timeout, 0))
.withHTTPKeepAliveMaxRequests(client_configuration.http_keep_alive_max_requests);
}
PocoHTTPClient::PocoHTTPClient(const PocoHTTPClientConfiguration & client_configuration)

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