From 82a849ba8e1b95833d2ae7f3bcdd3e1c008bacb5 Mon Sep 17 00:00:00 2001
From: Artur <613623@mail.ru>
Date: Sat, 18 Sep 2021 09:36:02 +0000
Subject: [PATCH 001/264] add options method
---
programs/server/config.xml | 19 ++++++++++++++++++
src/Interpreters/ClientInfo.h | 5 +++--
src/Server/HTTPHandler.cpp | 37 +++++++++++++++++++++++++++++++++++
3 files changed, 59 insertions(+), 2 deletions(-)
diff --git a/programs/server/config.xml b/programs/server/config.xml
index 6c98ac740fe..bec51de6126 100644
--- a/programs/server/config.xml
+++ b/programs/server/config.xml
@@ -62,6 +62,25 @@
-->
+
+
+ Access-Control-Allow-Origin
+ *
+
+
+ Access-Control-Allow-Headers
+ origin, x-requested-with
+
+
+ Access-Control-Allow-Methods
+ POST, GET, OPTIONS
+
+
+ Access-Control-Max-Age
+ 86400
+
+
+
diff --git a/src/Interpreters/ClientInfo.h b/src/Interpreters/ClientInfo.h
index 71570778645..294bf3b426c 100644
--- a/src/Interpreters/ClientInfo.h
+++ b/src/Interpreters/ClientInfo.h
@@ -35,8 +35,9 @@ public:
enum class HTTPMethod : uint8_t
{
UNKNOWN = 0,
- GET = 1,
- POST = 2,
+ GET = 1,
+ POST = 2,
+ OPTIONS = 3
};
enum class QueryKind : uint8_t
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index 0492b58dc88..99502261aa9 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -32,6 +32,7 @@
#include
#include
#include
+#include "Server/HTTP/HTTPResponse.h"
#if !defined(ARCADIA_BUILD)
# include
@@ -108,6 +109,37 @@ namespace ErrorCodes
extern const int HTTP_LENGTH_REQUIRED;
}
+namespace
+{
+ /// Process options request. Usefull for CORS.
+ void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config)
+ {
+ /// If answer for options request was not defined, return 501 to client.
+ if (!config.has("http_options_response"))
+ {
+ response.setStatusAndReason(HTTPResponse::HTTP_NOT_IMPLEMENTED);
+ response.send();
+ }
+ else
+ {
+ /// otherwise fill response.
+ Strings config_keys;
+ config.keys("http_options_response", config_keys);
+ for (const std::string & config_key : config_keys)
+ {
+ if (config_key == "header" || config_key.starts_with("header["))
+ {
+ response.add(config.getString("http_options_response." + config_key + ".name", "Empty header"),
+ config.getString("http_options_response." + config_key + ".value", ""));
+ response.setKeepAlive(false);
+ }
+ }
+ response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT);
+ response.send();
+ }
+ }
+}
+
static String base64Decode(const String & encoded)
{
String decoded;
@@ -850,6 +882,11 @@ void HTTPHandler::handleRequest(HTTPServerRequest & request, HTTPServerResponse
try
{
+ if (request.getMethod() == HTTPServerRequest::HTTP_OPTIONS)
+ {
+ processOptionsRequest(response, server.config());
+ return;
+ }
response.setContentType("text/plain; charset=UTF-8");
response.set("X-ClickHouse-Server-Display-Name", server_display_name);
/// For keep-alive to work.
From 6277747e8aa22b75a788b3e97372678b5c5df756 Mon Sep 17 00:00:00 2001
From: Alexey
Date: Mon, 20 Sep 2021 05:37:18 +0000
Subject: [PATCH 002/264] First draft
---
.../functions/other-functions.md | 36 +++++++++++++++++++
1 file changed, 36 insertions(+)
diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md
index 11cd522c622..af17954fec7 100644
--- a/docs/en/sql-reference/functions/other-functions.md
+++ b/docs/en/sql-reference/functions/other-functions.md
@@ -2354,3 +2354,39 @@ Result:
│ 1 │
└─────────┘
```
+
+## shardNum {#shard-num}
+
+Returns the number of a shard which executes the query for a distributed query.
+If query is not distributed then *constant value* is returned.
+
+**Syntax**
+
+``` sql
+shardNum()
+```
+
+**Returned value**
+
+- Shard number.
+
+Type: [UInt32](../../sql-reference/data-types/int-uint.md).
+
+## shardCount {#shard-count}
+
+Returns the total number of shards which execute a distributed query.
+If query is not distributed then *constant value* is returned.
+
+**Syntax**
+
+``` sql
+shardCount()
+```
+
+**Returned value**
+
+- Total number of shards.
+
+Type: [UInt32](../../sql-reference/data-types/int-uint.md).
+
+
From c8892ec7a71eac73a852ab1b8d200a86148b08c5 Mon Sep 17 00:00:00 2001
From: Artur <613623@mail.ru>
Date: Wed, 22 Sep 2021 10:34:48 +0000
Subject: [PATCH 003/264] add options support
---
src/Server/HTTPHandler.cpp | 19 +++++++++++++------
src/Server/HTTPHandlerFactory.cpp | 4 ++--
src/Server/HTTPHandlerFactory.h | 6 ++++--
3 files changed, 19 insertions(+), 10 deletions(-)
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index 99502261aa9..017bc82a475 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -25,6 +25,7 @@
#include
#include
#include
+#include "common/logger_useful.h"
#include
#include
#include
@@ -111,10 +112,11 @@ namespace ErrorCodes
namespace
{
- /// Process options request. Usefull for CORS.
+ /// Process options request. Useful for CORS.
void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config)
{
- /// If answer for options request was not defined, return 501 to client.
+ /// If response for options request was not defined, return 501 to client.
+ /// TODO should it be here?
if (!config.has("http_options_response"))
{
response.setStatusAndReason(HTTPResponse::HTTP_NOT_IMPLEMENTED);
@@ -129,12 +131,17 @@ namespace
{
if (config_key == "header" || config_key.starts_with("header["))
{
- response.add(config.getString("http_options_response." + config_key + ".name", "Empty header"),
- config.getString("http_options_response." + config_key + ".value", ""));
- response.setKeepAlive(false);
+ /// If there is empty header name, it will not be processed and message about it will be in logs
+ if (config.getString("http_options_response." + config_key + ".name", "").empty())
+ LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed.");
+ else
+ response.add(config.getString("http_options_response." + config_key + ".name", ""),
+ config.getString("http_options_response." + config_key + ".value", ""));
+
}
}
- response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT);
+ response.setKeepAlive(false);
+ response.setStatusAndReason(HTTPResponse::HTTP_OK);
response.send();
}
}
diff --git a/src/Server/HTTPHandlerFactory.cpp b/src/Server/HTTPHandlerFactory.cpp
index 1e3d02b85ab..526b86a5c28 100644
--- a/src/Server/HTTPHandlerFactory.cpp
+++ b/src/Server/HTTPHandlerFactory.cpp
@@ -123,7 +123,7 @@ static inline HTTPRequestHandlerFactoryPtr createInterserverHTTPHandlerFactory(I
addCommonDefaultHandlersFactory(*factory, server);
auto main_handler = std::make_shared>(server);
- main_handler->allowPostAndGetParamsRequest();
+ main_handler->allowPostAndGetParamsAndOptionsRequest();
factory->addHandler(main_handler);
return factory;
@@ -180,7 +180,7 @@ void addDefaultHandlersFactory(HTTPRequestHandlerFactoryMain & factory, IServer
addCommonDefaultHandlersFactory(factory, server);
auto query_handler = std::make_shared>(server, "query");
- query_handler->allowPostAndGetParamsRequest();
+ query_handler->allowPostAndGetParamsAndOptionsRequest();
factory.addHandler(query_handler);
/// We check that prometheus handler will be served on current (default) port.
diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h
index 6297f988eaa..5497d585d43 100644
--- a/src/Server/HTTPHandlerFactory.h
+++ b/src/Server/HTTPHandlerFactory.h
@@ -104,11 +104,13 @@ public:
}
/// Handle POST or GET with params
- void allowPostAndGetParamsRequest()
+ void allowPostAndGetParamsAndOptionsRequest()
{
addFilter([](const auto & request)
{
- return request.getURI().find('?') != std::string::npos
+ return (request.getURI().find('?') != std::string::npos
+ && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
+ || request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST;
});
}
From 2cffa98a60677517a372f72a1fea746a19bb0328 Mon Sep 17 00:00:00 2001
From: Artur <613623@mail.ru>
Date: Wed, 22 Sep 2021 12:22:21 +0000
Subject: [PATCH 004/264] add test and comments in config
---
programs/server/config.xml | 6 ++++--
src/Server/HTTPHandler.cpp | 14 ++++---------
tests/config/config.d/CORS.xml | 20 +++++++++++++++++++
.../02029_test_options_requests.reference | 5 +++++
.../02029_test_options_requests.sh | 8 ++++++++
5 files changed, 41 insertions(+), 12 deletions(-)
create mode 100644 tests/config/config.d/CORS.xml
create mode 100644 tests/queries/0_stateless/02029_test_options_requests.reference
create mode 100755 tests/queries/0_stateless/02029_test_options_requests.sh
diff --git a/programs/server/config.xml b/programs/server/config.xml
index bec51de6126..26c3107e972 100644
--- a/programs/server/config.xml
+++ b/programs/server/config.xml
@@ -62,7 +62,9 @@
-->
-
+
+
+
random
+
+
+ 1
diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
index a4fe3649e6f..5bc10841726 100644
--- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
+++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.cpp
@@ -29,9 +29,6 @@ void WriteBufferFromHTTPServerResponse::startSendHeaders()
{
headers_started_sending = true;
- if (add_cors_header)
- response.set("Access-Control-Allow-Origin", "*");
-
setResponseDefaultHeaders(response, keep_alive_timeout);
if (!is_http_method_head)
diff --git a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h
index b4ff454195f..7cebf5ca770 100644
--- a/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h
+++ b/src/Server/HTTP/WriteBufferFromHTTPServerResponse.h
@@ -36,7 +36,6 @@ private:
HTTPServerResponse & response;
bool is_http_method_head;
- bool add_cors_header = false;
unsigned keep_alive_timeout = 0;
bool compress = false;
CompressionMethod compression_method;
@@ -103,13 +102,6 @@ public:
compression_level = level;
}
- /// Turn CORS on or off.
- /// The setting has any effect only if HTTP headers haven't been sent yet.
- void addHeaderCORS(bool enable_cors)
- {
- add_cors_header = enable_cors;
- }
-
/// Don't send HTTP headers with progress more frequently.
void setSendProgressInterval(size_t send_progress_interval_ms_)
{
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index cec7e1c8b3d..c27d5343e90 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -33,7 +33,7 @@
#include
#include
#include
-#include "Server/HTTP/HTTPResponse.h"
+#include
#if !defined(ARCADIA_BUILD)
# include
@@ -112,33 +112,41 @@ namespace ErrorCodes
namespace
{
- /// Process options request. Useful for CORS.
- void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config)
+bool tryAddHeadersFromConfig(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config)
+{
+ if (config.has("http_options_response"))
{
- /// If there is information for options request in cofing, fill response.
- /// For this purpose find all headers related to http_options_response and add them with their values to response
- if (config.has("http_options_response"))
+ Strings config_keys;
+ config.keys("http_options_response", config_keys);
+ for (const std::string & config_key : config_keys)
{
- Strings config_keys;
- config.keys("http_options_response", config_keys);
- for (const std::string & config_key : config_keys)
+ if (config_key == "header" || config_key.starts_with("header["))
{
- if (config_key == "header" || config_key.starts_with("header["))
- {
- /// If there is empty header name, it will not be processed and message about it will be in logs
- if (config.getString("http_options_response." + config_key + ".name", "").empty())
- LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed.");
- else
- response.add(config.getString("http_options_response." + config_key + ".name", ""),
- config.getString("http_options_response." + config_key + ".value", ""));
+ /// If there is empty header name, it will not be processed and message about it will be in logs
+ if (config.getString("http_options_response." + config_key + ".name", "").empty())
+ LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed.");
+ else
+ response.add(config.getString("http_options_response." + config_key + ".name", ""),
+ config.getString("http_options_response." + config_key + ".value", ""));
- }
}
- response.setKeepAlive(false);
- response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT);
- response.send();
}
+ return true;
}
+ return false;
+}
+
+/// Process options request. Useful for CORS.
+void processOptionsRequest(HTTPServerResponse & response, const Poco::Util::LayeredConfiguration & config)
+{
+ /// If can add some headers from config
+ if (tryAddHeadersFromConfig(response, config))
+ {
+ response.setKeepAlive(false);
+ response.setStatusAndReason(HTTPResponse::HTTP_NO_CONTENT);
+ response.send();
+ }
+}
}
static String base64Decode(const String & encoded)
@@ -739,9 +747,10 @@ void HTTPHandler::processQuery(
if (in_post_compressed && settings.http_native_compression_disable_checksumming_on_decompress)
static_cast(*in_post_maybe_compressed).disableChecksumming();
- /// Add CORS header if 'add_http_cors_header' setting is turned on and the client passed
- /// Origin header.
- used_output.out->addHeaderCORS(settings.add_http_cors_header && !request.get("Origin", "").empty());
+ /// Add CORS header if 'add_http_cors_header' setting is turned on or config has http_options_response,
+ /// which means that there are some headers to be sent, and the client passed Origin header.
+ if ((settings.add_http_cors_header || config.has("http_options_response")) && !request.get("Origin", "").empty())
+ tryAddHeadersFromConfig(response, config);
auto append_callback = [context] (ProgressCallback callback)
{
From ce4193fe957367d28da59e6c94fc54aefb3038db Mon Sep 17 00:00:00 2001
From: Artur <613623@mail.ru>
Date: Wed, 22 Sep 2021 17:10:15 +0000
Subject: [PATCH 008/264] small refactoring
---
src/Server/HTTPHandler.cpp | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index c27d5343e90..046e7bdfaad 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -749,7 +749,7 @@ void HTTPHandler::processQuery(
/// Add CORS header if 'add_http_cors_header' setting is turned on or config has http_options_response,
/// which means that there are some headers to be sent, and the client passed Origin header.
- if ((settings.add_http_cors_header || config.has("http_options_response")) && !request.get("Origin", "").empty())
+ if (settings.add_http_cors_header && config.has("http_options_response") && !request.get("Origin", "").empty())
tryAddHeadersFromConfig(response, config);
auto append_callback = [context] (ProgressCallback callback)
From 7bbd08cb5d4c90357fc23b0cbfe96f36cfecff33 Mon Sep 17 00:00:00 2001
From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com>
Date: Fri, 24 Sep 2021 15:40:27 +0300
Subject: [PATCH 009/264] Update HTTPHandler.cpp
---
src/Server/HTTPHandler.cpp | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index bd4452ac6cb..7357c56ad2e 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -125,7 +125,7 @@ bool tryAddHeadersFromConfig(HTTPServerResponse & response, const Poco::Util::La
/// If there is empty header name, it will not be processed and message about it will be in logs
if (config.getString("http_options_response." + config_key + ".name", "").empty())
LOG_WARNING(&Poco::Logger::get("processOptionsRequest"), "Empty header was found in config. It will not be processed.");
- else
+ else
response.add(config.getString("http_options_response." + config_key + ".name", ""),
config.getString("http_options_response." + config_key + ".value", ""));
From b3325772f78e5b91ca6aaaee9cc1dd9beafda089 Mon Sep 17 00:00:00 2001
From: Viachaslau Boben
Date: Mon, 6 Sep 2021 02:25:22 +0300
Subject: [PATCH 010/264] Add normalizeUTF8 function with NFC normalization
---
src/Common/ErrorCodes.cpp | 1 +
src/Functions/normalizeString.cpp | 126 ++++++++++++++++++
src/Functions/registerFunctionsString.cpp | 8 ++
.../02011_normalize_utf8.reference | 3 +
.../0_stateless/02011_normalize_utf8.sql | 19 +++
5 files changed, 157 insertions(+)
create mode 100644 src/Functions/normalizeString.cpp
create mode 100644 tests/queries/0_stateless/02011_normalize_utf8.reference
create mode 100644 tests/queries/0_stateless/02011_normalize_utf8.sql
diff --git a/src/Common/ErrorCodes.cpp b/src/Common/ErrorCodes.cpp
index 53276f5b196..b6d9b65c28b 100644
--- a/src/Common/ErrorCodes.cpp
+++ b/src/Common/ErrorCodes.cpp
@@ -588,6 +588,7 @@
M(618, LZ4_DECODER_FAILED) \
M(619, POSTGRESQL_REPLICATION_INTERNAL_ERROR) \
M(620, QUERY_NOT_ALLOWED) \
+ M(621, CANNOT_NORMALIZE_STRING) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \
diff --git a/src/Functions/normalizeString.cpp b/src/Functions/normalizeString.cpp
new file mode 100644
index 00000000000..178c2dc2cf1
--- /dev/null
+++ b/src/Functions/normalizeString.cpp
@@ -0,0 +1,126 @@
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include
+#include "common/logger_useful.h"
+#include "Columns/ColumnString.h"
+#include "Parsers/IAST_fwd.h"
+
+namespace DB
+{
+
+namespace ErrorCodes
+{
+ extern const int CANNOT_NORMALIZE_STRING;
+}
+
+namespace
+{
+
+struct NormalizeUTF8Impl
+{
+
+ static void vector(const ColumnString::Chars & data,
+ const ColumnString::Offsets & offsets,
+ ColumnString::Chars & res_data,
+ ColumnString::Offsets & res_offsets)
+ {
+ UErrorCode err = U_ZERO_ERROR;
+
+ const UNormalizer2 *normalizer = unorm2_getNFCInstance(&err);
+ if (U_FAILURE(err)) {
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
+ }
+
+ size_t size = offsets.size();
+ res_offsets.resize(size);
+
+ ColumnString::Offset current_from_offset = 0;
+ ColumnString::Offset current_to_offset = 0;
+
+ icu::UnicodeString to_string;
+
+ PODArray from_uchars;
+ PODArray to_uchars;
+
+ for (size_t i = 0; i < size; ++i)
+ {
+ size_t from_size = offsets[i] - current_from_offset - 1;
+
+ from_uchars.resize(from_size + 1);
+ int32_t from_code_points;
+ u_strFromUTF8(
+ from_uchars.data(),
+ from_uchars.size(),
+ &from_code_points,
+ reinterpret_cast(&data[current_from_offset]),
+ from_size,
+ &err);
+ if (U_FAILURE(err)) {
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
+ }
+
+ // NFC should produce no more than 3x code points
+ // https://unicode.org/faq/normalization.html#12
+ to_uchars.resize(from_code_points * 3 + 1);
+
+ int32_t to_code_points = unorm2_normalize(
+ normalizer,
+ from_uchars.data(),
+ from_code_points,
+ to_uchars.data(),
+ to_uchars.size(),
+ &err);
+ if (U_FAILURE(err)) {
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
+ }
+
+ size_t max_to_size = current_to_offset + 2 * to_code_points + 1;
+ if (res_data.size() < max_to_size) {
+ res_data.resize(max_to_size);
+ }
+
+ int32_t to_size;
+ u_strToUTF8(
+ reinterpret_cast(&res_data[current_to_offset]),
+ res_data.size() - current_to_offset,
+ &to_size,
+ to_uchars.data(),
+ to_code_points,
+ &err);
+ if (U_FAILURE(err)) {
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
+ }
+
+ current_to_offset += to_size;
+ res_data[current_to_offset] = 0;
+ ++current_to_offset;
+ res_offsets[i] = current_to_offset;
+
+ current_from_offset = offsets[i];
+ }
+ }
+
+ [[noreturn]] static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &)
+ {
+ throw Exception("Cannot apply function normalizeUTF8 to fixed string.", ErrorCodes::ILLEGAL_COLUMN);
+ }
+};
+
+struct NameNormalizeUTF8
+{
+ static constexpr auto name = "normalizeUTF8";
+};
+
+using FunctionNormalizeUTF8 = FunctionStringToString;
+}
+
+void registerFunctionNormalizeUTF8(FunctionFactory & factory) {
+ factory.registerFunction();
+}
+
+}
diff --git a/src/Functions/registerFunctionsString.cpp b/src/Functions/registerFunctionsString.cpp
index ba6a294abba..f2439a3373b 100644
--- a/src/Functions/registerFunctionsString.cpp
+++ b/src/Functions/registerFunctionsString.cpp
@@ -52,6 +52,10 @@ void registerFunctionSynonyms(FunctionFactory &);
void registerFunctionLemmatize(FunctionFactory &);
#endif
+#if USE_ICU
+void registerFunctionNormalizeUTF8(FunctionFactory &);
+#endif
+
void registerFunctionsString(FunctionFactory & factory)
{
registerFunctionRepeat(factory);
@@ -97,6 +101,10 @@ void registerFunctionsString(FunctionFactory & factory)
registerFunctionSynonyms(factory);
registerFunctionLemmatize(factory);
#endif
+
+#if USE_ICU
+ registerFunctionNormalizeUTF8(factory);
+#endif
}
}
diff --git a/tests/queries/0_stateless/02011_normalize_utf8.reference b/tests/queries/0_stateless/02011_normalize_utf8.reference
new file mode 100644
index 00000000000..6878a38ca0d
--- /dev/null
+++ b/tests/queries/0_stateless/02011_normalize_utf8.reference
@@ -0,0 +1,3 @@
+ё ё 2 4 ё ё 2 2
+ё 4 ё 2
+ё 2 ё 2
diff --git a/tests/queries/0_stateless/02011_normalize_utf8.sql b/tests/queries/0_stateless/02011_normalize_utf8.sql
new file mode 100644
index 00000000000..c28a0c0a794
--- /dev/null
+++ b/tests/queries/0_stateless/02011_normalize_utf8.sql
@@ -0,0 +1,19 @@
+DROP TABLE IF EXISTS normalize_test;
+CREATE TABLE normalize_test (value String) ENGINE = MergeTree ORDER BY value;
+
+SELECT
+ 'ё' AS norm,
+ 'ё' AS denorm,
+ length(norm),
+ length(denorm),
+ normalizeUTF8(norm),
+ normalizeUTF8(denorm),
+ length(normalizeUTF8(norm)),
+ length(normalizeUTF8(denorm));
+
+INSERT INTO normalize_test (value) VALUES ('ё');
+INSERT INTO normalize_test (value) VALUES ('ё');
+
+SELECT value, length(value), normalizeUTF8(value) AS normalized, length(normalized) FROM normalize_test;
+
+SELECT char(228) AS value, normalizeUTF8(value); -- { serverError 619 }
From 762904adbda95dc24b771250b1f32ccd404db739 Mon Sep 17 00:00:00 2001
From: Viachaslau Boben
Date: Mon, 27 Sep 2021 18:45:04 +0300
Subject: [PATCH 011/264] Add nfd and perf test
---
src/Functions/normalizeString.cpp | 114 +++++++++++++-----
tests/performance/normalize_utf8.xml | 15 +++
.../02011_normalize_utf8.reference | 12 +-
.../0_stateless/02011_normalize_utf8.sql | 51 ++++++--
4 files changed, 148 insertions(+), 44 deletions(-)
create mode 100644 tests/performance/normalize_utf8.xml
diff --git a/src/Functions/normalizeString.cpp b/src/Functions/normalizeString.cpp
index 178c2dc2cf1..5beca566cd1 100644
--- a/src/Functions/normalizeString.cpp
+++ b/src/Functions/normalizeString.cpp
@@ -1,6 +1,10 @@
+#if !defined(ARCADIA_BUILD)
+# include "config_core.h"
+#endif
+
+#if USE_ICU
#include
#include
-#include
#include
#include
#include
@@ -15,12 +19,67 @@ namespace DB
namespace ErrorCodes
{
+ extern const int ILLEGAL_COLUMN;
extern const int CANNOT_NORMALIZE_STRING;
}
namespace
{
+// Expansion factors are specified for UTF-32, since icu uses UTF-32 for normalization
+// Maximum expansion factors for different normalization forms
+// https://unicode.org/faq/normalization.html#12
+
+struct NormalizeNFCImpl
+{
+ static constexpr auto name = "normalizeUTF8NFC";
+
+ static constexpr auto expansionFactor = 3;
+
+ static const UNormalizer2 *getNormalizer(UErrorCode *err)
+ {
+ return unorm2_getNFCInstance(err);
+ }
+};
+
+struct NormalizeNFDImpl
+{
+ static constexpr auto name = "normalizeUTF8NFD";
+
+ static constexpr auto expansionFactor = 4;
+
+ static const UNormalizer2 *getNormalizer(UErrorCode *err)
+ {
+ return unorm2_getNFDInstance(err);
+ }
+};
+
+struct NormalizeNFKCImpl
+{
+ static constexpr auto name = "normalizeUTF8NFKC";
+
+ static constexpr auto expansionFactor = 18;
+
+ static const UNormalizer2 *getNormalizer(UErrorCode *err)
+ {
+ return unorm2_getNFKCInstance(err);
+ }
+};
+
+
+struct NormalizeNFKDImpl
+{
+ static constexpr auto name = "normalizeUTF8NFKD";
+
+ static constexpr auto expansionFactor = 18;
+
+ static const UNormalizer2 *getNormalizer(UErrorCode *err)
+ {
+ return unorm2_getNFKDInstance(err);
+ }
+};
+
+template
struct NormalizeUTF8Impl
{
@@ -31,10 +90,9 @@ struct NormalizeUTF8Impl
{
UErrorCode err = U_ZERO_ERROR;
- const UNormalizer2 *normalizer = unorm2_getNFCInstance(&err);
- if (U_FAILURE(err)) {
- throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
- }
+ const UNormalizer2 *normalizer = NormalizeImpl::getNormalizer(&err);
+ if (U_FAILURE(err))
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (getNormalizer): {}", u_errorName(err));
size_t size = offsets.size();
res_offsets.resize(size);
@@ -60,13 +118,10 @@ struct NormalizeUTF8Impl
reinterpret_cast(&data[current_from_offset]),
from_size,
&err);
- if (U_FAILURE(err)) {
- throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
- }
+ if (U_FAILURE(err))
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (strFromUTF8): {}", u_errorName(err));
- // NFC should produce no more than 3x code points
- // https://unicode.org/faq/normalization.html#12
- to_uchars.resize(from_code_points * 3 + 1);
+ to_uchars.resize(from_code_points * NormalizeImpl::expansionFactor + 1);
int32_t to_code_points = unorm2_normalize(
normalizer,
@@ -75,14 +130,12 @@ struct NormalizeUTF8Impl
to_uchars.data(),
to_uchars.size(),
&err);
- if (U_FAILURE(err)) {
- throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
- }
+ if (U_FAILURE(err))
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (normalize): {}", u_errorName(err));
- size_t max_to_size = current_to_offset + 2 * to_code_points + 1;
- if (res_data.size() < max_to_size) {
+ size_t max_to_size = current_to_offset + 4 * to_code_points + 1;
+ if (res_data.size() < max_to_size)
res_data.resize(max_to_size);
- }
int32_t to_size;
u_strToUTF8(
@@ -92,9 +145,8 @@ struct NormalizeUTF8Impl
to_uchars.data(),
to_code_points,
&err);
- if (U_FAILURE(err)) {
- throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed: {}", u_errorName(err));
- }
+ if (U_FAILURE(err))
+ throw Exception(ErrorCodes::CANNOT_NORMALIZE_STRING, "Normalization failed (strToUTF8): {}", u_errorName(err));
current_to_offset += to_size;
res_data[current_to_offset] = 0;
@@ -111,16 +163,20 @@ struct NormalizeUTF8Impl
}
};
-struct NameNormalizeUTF8
+using FunctionNormalizeUTF8NFC = FunctionStringToString, NormalizeNFCImpl>;
+using FunctionNormalizeUTF8NFD = FunctionStringToString, NormalizeNFDImpl>;
+using FunctionNormalizeUTF8NFKC = FunctionStringToString, NormalizeNFKCImpl>;
+using FunctionNormalizeUTF8NFKD = FunctionStringToString, NormalizeNFKDImpl>;
+}
+
+void registerFunctionNormalizeUTF8(FunctionFactory & factory)
{
- static constexpr auto name = "normalizeUTF8";
-};
-
-using FunctionNormalizeUTF8 = FunctionStringToString;
-}
-
-void registerFunctionNormalizeUTF8(FunctionFactory & factory) {
- factory.registerFunction();
+ factory.registerFunction();
+ factory.registerFunction();
+ factory.registerFunction();
+ factory.registerFunction();
}
}
+
+#endif
diff --git a/tests/performance/normalize_utf8.xml b/tests/performance/normalize_utf8.xml
new file mode 100644
index 00000000000..de9bd87fdf8
--- /dev/null
+++ b/tests/performance/normalize_utf8.xml
@@ -0,0 +1,15 @@
+
+
+ hits_10m_single
+
+
+ CREATE TABLE strings (words String) ENGINE Memory
+ INSERT INTO strings SELECT SearchPhrase FROM hits_10m_single WHERE length(SearchPhrase) > 0
+
+ SELECT normalizeUTF8NFC(words) FROM strings FORMAT Null
+ SELECT normalizeUTF8NFD(words) FROM strings FORMAT Null
+ SELECT normalizeUTF8NFKC(words) FROM strings FORMAT Null
+ SELECT normalizeUTF8NFKD(words) FROM strings FORMAT Null
+
+ DROP TABLE IF EXISTS strings
+
diff --git a/tests/queries/0_stateless/02011_normalize_utf8.reference b/tests/queries/0_stateless/02011_normalize_utf8.reference
index 6878a38ca0d..b97f0ee5a01 100644
--- a/tests/queries/0_stateless/02011_normalize_utf8.reference
+++ b/tests/queries/0_stateless/02011_normalize_utf8.reference
@@ -1,3 +1,11 @@
ё ё 2 4 ё ё 2 2
-ё 4 ё 2
-ё 2 ё 2
+1 ё 4 ё 2 ё 4 ё 2 ё 4
+2 ё 2 ё 2 ё 4 ё 2 ё 4
+3 జ్ఞా 15 జ్ఞా 15 జ్ఞా 15 జ్ఞా 15 జ్ఞా 15
+4 本気ですか 15 本気ですか 15 本気ですか 18 本気ですか 15 本気ですか 18
+5 ﷺ 3 ﷺ 3 ﷺ 3 صلى الله عليه وسلم 33 صلى الله عليه وسلم 33
+6 ᾂ 3 ᾂ 3 ᾂ 8 ᾂ 3 ᾂ 8
+7 ΐ 2 ΐ 2 ΐ 6 ΐ 2 ΐ 6
+8 שּׁ 6 שּׁ 6 שּׁ 6 שּׁ 6 שּׁ 6
+9 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12 𝅘𝅥𝅮 12
+10 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 281 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 281 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 282 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 281 Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒ 282
diff --git a/tests/queries/0_stateless/02011_normalize_utf8.sql b/tests/queries/0_stateless/02011_normalize_utf8.sql
index c28a0c0a794..5abb6b4d8fb 100644
--- a/tests/queries/0_stateless/02011_normalize_utf8.sql
+++ b/tests/queries/0_stateless/02011_normalize_utf8.sql
@@ -1,19 +1,44 @@
+-- Tags: no-fasttest
+
DROP TABLE IF EXISTS normalize_test;
-CREATE TABLE normalize_test (value String) ENGINE = MergeTree ORDER BY value;
+CREATE TABLE normalize_test (id int, value String) ENGINE = MergeTree ORDER BY value;
+
SELECT
- 'ё' AS norm,
- 'ё' AS denorm,
- length(norm),
- length(denorm),
- normalizeUTF8(norm),
- normalizeUTF8(denorm),
- length(normalizeUTF8(norm)),
- length(normalizeUTF8(denorm));
+ 'ё' AS norm, 'ё' AS denorm,
+ length(norm), length(denorm),
+ normalizeUTF8NFC(norm) AS norm_nfc,
+ normalizeUTF8NFC(denorm) AS denorm_nfc,
+ length(norm_nfc),
+ length(denorm_nfc);
-INSERT INTO normalize_test (value) VALUES ('ё');
-INSERT INTO normalize_test (value) VALUES ('ё');
-SELECT value, length(value), normalizeUTF8(value) AS normalized, length(normalized) FROM normalize_test;
+INSERT INTO normalize_test (id, value) VALUES (1, 'ё');
+INSERT INTO normalize_test (id, value) VALUES (2, 'ё');
+INSERT INTO normalize_test (id, value) VALUES (3, 'జ్ఞా');
+INSERT INTO normalize_test (id, value) VALUES (4, '本気ですか');
+INSERT INTO normalize_test (id, value) VALUES (5, 'ﷺ');
+INSERT INTO normalize_test (id, value) VALUES (6, 'ᾂ');
+INSERT INTO normalize_test (id, value) VALUES (7, 'ΐ');
+INSERT INTO normalize_test (id, value) VALUES (8, 'שּׁ');
+INSERT INTO normalize_test (id, value) VALUES (9, '𝅘𝅥𝅮');
-SELECT char(228) AS value, normalizeUTF8(value); -- { serverError 619 }
+
+INSERT INTO normalize_test (id, value) VALUES (10, 'Q̹̣̩̭̰̰̹̄ͬ̿͋̃ṷ̬̰ͥe̘͚͈̰̺̍͐s͎̜̖t͔̣̯̲̜̠ͣ̑ͨ̉̈̈o̲͙̺͊ͯͣ̐̋̂̔ ̳͉͍̒̂è̗ͥͯͨ̍ͮ͛ ̦̹̣̰̐̅̑͑̅̂t͙̭̻̖͛̾e̺͙ͣ͒̚ṣ̠͉͓͔̲̦̎t̖͖̝͓̣ͭ͑̈́̂ỏ̥͕͈͛̓ ̀ͦ̽ͅZͯ̑̎a͆l̻ͨ̋ͧͣͨͬg͉̙̟̾̅̾ͬo̠ͮ͒');
+
+
+
+SELECT
+ id, value, length(value),
+ normalizeUTF8NFC(value) AS nfc, length(nfc) AS nfc_len,
+ normalizeUTF8NFD(value) AS nfd, length(nfd) AS nfd_len,
+ normalizeUTF8NFKC(value) AS nfkc, length(nfkc) AS nfkc_len,
+ normalizeUTF8NFKD(value) AS nfkd, length(nfkd) AS nfkd_len
+FROM normalize_test
+ORDER BY id;
+
+
+SELECT char(228) AS value, normalizeUTF8NFC(value); -- { serverError 621 }
+SELECT char(228) AS value, normalizeUTF8NFD(value); -- { serverError 621 }
+SELECT char(228) AS value, normalizeUTF8NFKC(value); -- { serverError 621 }
+SELECT char(228) AS value, normalizeUTF8NFKD(value); -- { serverError 621 }
From fcebf7b9853452caaffc39d91a31d19ae55a45ba Mon Sep 17 00:00:00 2001
From: Artur <613623@mail.ru>
Date: Wed, 29 Sep 2021 11:29:24 +0000
Subject: [PATCH 012/264] correct tests
---
src/Server/HTTPHandlerFactory.h | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
diff --git a/src/Server/HTTPHandlerFactory.h b/src/Server/HTTPHandlerFactory.h
index e81955ef2b2..f6d96189d92 100644
--- a/src/Server/HTTPHandlerFactory.h
+++ b/src/Server/HTTPHandlerFactory.h
@@ -108,8 +108,7 @@ public:
{
addFilter([](const auto & request)
{
- return (request.getURI().find('?') != std::string::npos
- && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET)
+ return request.getURI().find('?') != std::string::npos
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_OPTIONS
|| request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST;
});
From 36b699659e466c1deaf4737f973adcfc95fe378b Mon Sep 17 00:00:00 2001
From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com>
Date: Wed, 29 Sep 2021 14:32:04 +0300
Subject: [PATCH 013/264] Update CORS.xml
---
tests/config/config.d/CORS.xml | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/tests/config/config.d/CORS.xml b/tests/config/config.d/CORS.xml
index 9dd7d402416..873821478dc 100644
--- a/tests/config/config.d/CORS.xml
+++ b/tests/config/config.d/CORS.xml
@@ -1,4 +1,4 @@
-
+
Access-Control-Allow-Origin
@@ -17,5 +17,5 @@
86400
-
+
From d75136c3b1b3040b87dde90463e9a0e8a087b16b Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Wed, 29 Sep 2021 16:59:56 +0300
Subject: [PATCH 014/264] Update hash functions (SHA) en
---
.../sql-reference/functions/hash-functions.md | 145 +++++++++++++++++-
1 file changed, 144 insertions(+), 1 deletion(-)
diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md
index 227e2885417..a3154e5c200 100644
--- a/docs/en/sql-reference/functions/hash-functions.md
+++ b/docs/en/sql-reference/functions/hash-functions.md
@@ -139,17 +139,160 @@ It works faster than intHash32. Average quality.
## SHA1 {#sha1}
+Calculates SHA-1 hash from a string and returns the resulting set of bytes as [FixedString(20)](../data-types/fixedstring.md).
+
+**Syntax**
+
+``` sql
+SHA1('s')
+```
+
+**Arguments**
+
+- `s` — Input string for SHA-1 hash calculation. [String](..data-types/string.md).
+
+**Returned value**
+
+- SHA-1 hash as a hex-unencoded FixedString(10).
+
+Type: [FixedString](../data-types/fixedstring.md).
+
+**Example**
+
+Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string.
+
+Query:
+
+``` sql
+SELECT hex(SHA1('abc'));
+```
+
+Result:
+
+``` text
+┌─hex(SHA1('abc'))─────────────────────────┐
+│ A9993E364706816ABA3E25717850C26C9CD0D89D │
+└──────────────────────────────────────────┘
+```
+
## SHA224 {#sha224}
+Calculates SHA-224 hash from a string and returns the resulting set of bytes as [FixedString(28)](../data-types/fixedstring.md).
+
+**Syntax**
+
+``` sql
+SHA224('s')
+```
+
+**Arguments**
+
+- `s` — Input string for SHA-224 hash calculation. [String](..data-types/string.md).
+
+**Returned value**
+
+- SHA-224 hash as a hex-unencoded FixedString(28).
+
+Type: [FixedString](../data-types/fixedstring.md).
+
+**Example**
+
+Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string.
+
+Query:
+
+``` sql
+SELECT hex(SHA224('abc'));
+```
+
+Result:
+
+``` text
+┌─hex(SHA224('abc'))───────────────────────────────────────┐
+│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │
+└──────────────────────────────────────────────────────────┘
+```
+
## SHA256 {#sha256}
+Calculates SHA-256 hash from a string and returns the resulting set of bytes as [FixedString(32)](../data-types/fixedstring.md).
+
+**Syntax**
+
+``` sql
+SHA256('s')
+```
+
+**Arguments**
+
+- `s` — Input string for SHA-256 hash calculation. [String](..data-types/string.md).
+
+**Returned value**
+
+- SHA-256 hash as a hex-unencoded FixedString(32).
+
+Type: [FixedString](../data-types/fixedstring.md).
+
+**Example**
+
+Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string.
+
+Query:
+
+``` sql
+SELECT hex(SHA256('abc'));
+```
+
+Result:
+
+``` text
+┌─hex(SHA256('abc'))───────────────────────────────────────────────┐
+│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │
+└──────────────────────────────────────────────────────────────────┘
+```
+
## SHA512 {#sha512}
-Calculates SHA-1, SHA-224, SHA-256 or SHA-512 from a string and returns the resulting set of bytes as FixedString(20), FixedString(28), FixedString(32), or FixedString(64).
+Calculates SHA-512 hash from a string and returns the resulting set of bytes as [FixedString(64)](../data-types/fixedstring.md).
+
+**Syntax**
+
+``` sql
+SHA512('s')
+```
+
The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million).
We recommend using this function only in cases when you need a specific hash function and you can’t select it.
Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in SELECTS.
+**Arguments**
+
+- `s` — Input string for SHA-512 hash calculation. [String](..data-types/string.md).
+
+**Returned value**
+
+- SHA-512 hash as a hex-unencoded FixedString(64).
+
+Type: [FixedString](../data-types/fixedstring.md).
+
+**Example**
+
+Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string.
+
+Query:
+
+``` sql
+SELECT hex(SHA512('abc'));
+```
+
+Result:
+
+``` text
+┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
+│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │
+└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
+```
+
## URLHash(url\[, N\]) {#urlhashurl-n}
A fast, decent-quality non-cryptographic hash function for a string obtained from a URL using some type of normalization.
From e312156b1c2a6a8a79177fb543c5d110ea47a058 Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Wed, 29 Sep 2021 17:52:39 +0300
Subject: [PATCH 015/264] Add note about Materialized views
---
docs/en/sql-reference/functions/hash-functions.md | 2 +-
docs/en/sql-reference/statements/create/view.md | 7 +++----
docs/ru/sql-reference/statements/create/view.md | 5 ++++-
3 files changed, 8 insertions(+), 6 deletions(-)
diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md
index a3154e5c200..dc4c749865a 100644
--- a/docs/en/sql-reference/functions/hash-functions.md
+++ b/docs/en/sql-reference/functions/hash-functions.md
@@ -263,7 +263,7 @@ SHA512('s')
The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million).
We recommend using this function only in cases when you need a specific hash function and you can’t select it.
-Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in SELECTS.
+Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in `SELECT` queries.
**Arguments**
diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md
index b6a09e25f95..84213020925 100644
--- a/docs/en/sql-reference/statements/create/view.md
+++ b/docs/en/sql-reference/statements/create/view.md
@@ -50,14 +50,13 @@ When creating a materialized view with `TO [db].[table]`, you must not use `POPU
A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view.
!!! important "Important"
- Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not `Nullable`. A safe practice would be to add aliases for every column when using Materialized views.
+ Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views.
-!!! important "Important"
Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view.
-If you specify `POPULATE`, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using POPULATE, since data inserted in the table during the view creation will not be inserted in it.
+If you specify `POPULATE`, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it.
-A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`.
+A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`. Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`.
The execution of [ALTER](../../../sql-reference/statements/alter/view.md) queries on materialized views has limitations, so they might be inconvenient. If the materialized view uses the construction `TO [db.]name`, you can `DETACH` the view, run `ALTER` for the target table, and then `ATTACH` the previously detached (`DETACH`) view.
diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md
index ccbf79baa73..53d75b78dd1 100644
--- a/docs/ru/sql-reference/statements/create/view.md
+++ b/docs/ru/sql-reference/statements/create/view.md
@@ -48,9 +48,12 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na
Материализованное представление устроено следующим образом: при вставке данных в таблицу, указанную в SELECT-е, кусок вставляемых данных преобразуется этим запросом SELECT, и полученный результат вставляется в представление.
!!! important "Важно"
+
+ Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца.
+
Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление.
-Если указано `POPULATE`, то при создании представления, в него будут вставлены имеющиеся данные таблицы, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Иначе, представление будет содержать только данные, вставляемые в таблицу после создания представления. Не рекомендуется использовать POPULATE, так как вставляемые в таблицу данные во время создания представления, не попадут в него.
+Если указано `POPULATE`, то при создании представления, в него будут вставлены имеющиеся данные таблицы, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Иначе, представление будет содержать только данные, вставляемые в таблицу после создания представления. Не рекомендуется использовать `POPULATE`, так как вставляемые в таблицу данные во время создания представления, не попадут в него.
Запрос `SELECT` может содержать `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Следует иметь ввиду, что соответствующие преобразования будут выполняться независимо, на каждый блок вставляемых данных. Например, при наличии `GROUP BY`, данные будут агрегироваться при вставке, но только в рамках одной пачки вставляемых данных. Далее, данные не будут доагрегированы. Исключение - использование ENGINE, производящего агрегацию данных самостоятельно, например, `SummingMergeTree`.
From b226429435eeda0cae88b7553f471b6e413cff3d Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Wed, 29 Sep 2021 18:41:15 +0300
Subject: [PATCH 016/264] Fix links, add 512 translation.
---
.../sql-reference/functions/hash-functions.md | 8 ++--
.../sql-reference/functions/hash-functions.md | 45 ++++++++++++++++++-
2 files changed, 47 insertions(+), 6 deletions(-)
diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md
index dc4c749865a..e28594540be 100644
--- a/docs/en/sql-reference/functions/hash-functions.md
+++ b/docs/en/sql-reference/functions/hash-functions.md
@@ -149,7 +149,7 @@ SHA1('s')
**Arguments**
-- `s` — Input string for SHA-1 hash calculation. [String](..data-types/string.md).
+- `s` — Input string for SHA-1 hash calculation. [String](../data-types/string.md).
**Returned value**
@@ -187,7 +187,7 @@ SHA224('s')
**Arguments**
-- `s` — Input string for SHA-224 hash calculation. [String](..data-types/string.md).
+- `s` — Input string for SHA-224 hash calculation. [String](../data-types/string.md).
**Returned value**
@@ -225,7 +225,7 @@ SHA256('s')
**Arguments**
-- `s` — Input string for SHA-256 hash calculation. [String](..data-types/string.md).
+- `s` — Input string for SHA-256 hash calculation. [String](../data-types/string.md).
**Returned value**
@@ -267,7 +267,7 @@ Even in these cases, we recommend applying the function offline and pre-calculat
**Arguments**
-- `s` — Input string for SHA-512 hash calculation. [String](..data-types/string.md).
+- `s` — Input string for SHA-512 hash calculation. [String](../data-types/string.md).
**Returned value**
diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md
index 07c741e0588..d7e86d5a540 100644
--- a/docs/ru/sql-reference/functions/hash-functions.md
+++ b/docs/ru/sql-reference/functions/hash-functions.md
@@ -143,10 +143,51 @@ SELECT groupBitXor(cityHash64(*)) FROM table
## SHA256 {#sha256}
-Вычисляет SHA-1, SHA-224, SHA-256 от строки и возвращает полученный набор байт в виде FixedString(20), FixedString(28), FixedString(32).
+
+
+## SHA512 {#sha512}
+
+Вычисляет SHA-1, SHA-224, SHA-256 хеш строки и возвращает полученный набор байт в виде FixedString(20), FixedString(28), FixedString(32), [FixedString(64)](../data-types/fixedstring.md)
+
+Вычисляет SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString(64)](../data-types/fixedstring.md)
+
+**Синтаксис**
+
+``` sql
+SHA512('s')
+```
+
Функция работает достаточно медленно (SHA-1 - примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 - примерно 2.2 миллионов).
Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать.
-Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при SELECT-ах.
+Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при выполнении `SELECT`.
+
+**Параметры**
+
+- `s` — входная строка для вычисления хеша SHA-512. [String](../data-types/string.md).
+
+**Возвращаемое значение**
+
+- Хеш SHA-512 в виде шестнадцатеричной некодированной строки FixedString(64).
+
+Тип: [FixedString](../data-types/fixedstring.md).
+
+**Пример**
+
+Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой.
+
+Запрос:
+
+``` sql
+SELECT hex(SHA512('abc'));
+```
+
+Результат:
+
+``` text
+┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
+│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │
+└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
+```
## URLHash(url\[, N\]) {#urlhashurl-n}
From cac28833d247617804627e3059589da17c09de1d Mon Sep 17 00:00:00 2001
From: Artur Filatenkov <613623@mail.ru>
Date: Wed, 29 Sep 2021 18:54:04 +0300
Subject: [PATCH 017/264] apply added config in tests
---
tests/config/install.sh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tests/config/install.sh b/tests/config/install.sh
index df62cba0ea9..936c44a4e7b 100755
--- a/tests/config/install.sh
+++ b/tests/config/install.sh
@@ -37,6 +37,7 @@ ln -sf $SRC_PATH/config.d/tcp_with_proxy.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/top_level_domains_lists.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/top_level_domains_path.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/encryption.xml $DEST_SERVER_PATH/config.d/
+ln -sf $SRC_PATH/config.d/CORS.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/zookeeper_log.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/config.d/logger.xml $DEST_SERVER_PATH/config.d/
ln -sf $SRC_PATH/users.d/log_queries.xml $DEST_SERVER_PATH/users.d/
@@ -57,7 +58,6 @@ ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/executable_pool_dictionary.xml $DEST_SERVER_PATH/
-ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/top_level_domains $DEST_SERVER_PATH/
From 02205492e5cef5119455bbed48d339349cb4575e Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Wed, 29 Sep 2021 19:55:11 +0300
Subject: [PATCH 018/264] Update hash-functions.md
Add ru translation.
---
.../sql-reference/functions/hash-functions.md | 110 +++++++++++++++++-
1 file changed, 107 insertions(+), 3 deletions(-)
diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md
index d7e86d5a540..98b5ed6df27 100644
--- a/docs/ru/sql-reference/functions/hash-functions.md
+++ b/docs/ru/sql-reference/functions/hash-functions.md
@@ -139,16 +139,120 @@ SELECT groupBitXor(cityHash64(*)) FROM table
## SHA1 {#sha1}
+Вычисляет SHA-1 хеш строки и возвращает полученный набор байт в виде [FixedString(20)](../data-types/fixedstring.md).
+
+**Синтаксис**
+
+``` sql
+SHA1('s')
+```
+
+**Параметры**
+
+- `s` — входная строка для вычисления хеша SHA-1. [String](../data-types/string.md).
+
+**Возвращаемое значение**
+
+- Хеш SHA-1 в виде шестнадцатеричной некодированной строки FixedString(20).
+
+Тип: [FixedString](../data-types/fixedstring.md).
+
+**Пример**
+
+Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой.
+
+Запрос:
+
+``` sql
+SELECT hex(SHA1('abc'));
+```
+
+Результат:
+
+``` text
+┌─hex(SHA1('abc'))─────────────────────────┐
+│ A9993E364706816ABA3E25717850C26C9CD0D89D │
+└──────────────────────────────────────────┘
+```
+
## SHA224 {#sha224}
+Вычисляет SHA-224 хеш строки и возвращает полученный набор байт в виде [FixedString(28)](../data-types/fixedstring.md).
+
+**Синтаксис**
+
+``` sql
+SHA224('s')
+```
+
+**Параметры**
+
+- `s` — входная строка для вычисления хеша SHA-224. [String](../data-types/string.md).
+
+**Возвращаемое значение**
+
+- Хеш SHA-224 в виде шестнадцатеричной некодированной строки FixedString(28).
+
+Тип: [FixedString](../data-types/fixedstring.md).
+
+**Пример**
+
+Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой.
+
+Запрос:
+
+``` sql
+SELECT hex(SHA224('abc'));
+```
+
+Результат:
+
+``` text
+┌─hex(SHA224('abc'))───────────────────────────────────────┐
+│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │
+└──────────────────────────────────────────────────────────┘
+```
+
## SHA256 {#sha256}
+Вычисляет SHA-256 хеш строки и возвращает полученный набор байт в виде [FixedString(32)](../data-types/fixedstring.md).
+**Синтаксис**
+
+``` sql
+SHA256('s')
+```
+
+**Параметры**
+
+- `s` — входная строка для вычисления хеша SHA-256. [String](../data-types/string.md).
+
+**Возвращаемое значение**
+
+- Хеш SHA-256 в виде шестнадцатеричной некодированной строки FixedString(32).
+
+Тип: [FixedString](../data-types/fixedstring.md).
+
+**Пример**
+
+Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой.
+
+Запрос:
+
+``` sql
+SELECT hex(SHA256('abc'));
+```
+
+Результат:
+
+``` text
+┌─hex(SHA256('abc'))───────────────────────────────────────────────┐
+│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │
+└──────────────────────────────────────────────────────────────────┘
+```
## SHA512 {#sha512}
-Вычисляет SHA-1, SHA-224, SHA-256 хеш строки и возвращает полученный набор байт в виде FixedString(20), FixedString(28), FixedString(32), [FixedString(64)](../data-types/fixedstring.md)
-
Вычисляет SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString(64)](../data-types/fixedstring.md)
**Синтаксис**
@@ -157,7 +261,7 @@ SELECT groupBitXor(cityHash64(*)) FROM table
SHA512('s')
```
-Функция работает достаточно медленно (SHA-1 - примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 - примерно 2.2 миллионов).
+Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов).
Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать.
Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при выполнении `SELECT`.
From 66bb857a1a7988ec0f94bc9c83668ff662b651b5 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?=
Date: Wed, 29 Sep 2021 19:11:38 +0200
Subject: [PATCH 019/264] Add test for JOIN engine deadlock
---
.../02033_join_engine_deadlock.reference | 0
.../0_stateless/02033_join_engine_deadlock.sh | 71 +++++++++++++++++++
2 files changed, 71 insertions(+)
create mode 100644 tests/queries/0_stateless/02033_join_engine_deadlock.reference
create mode 100755 tests/queries/0_stateless/02033_join_engine_deadlock.sh
diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.reference b/tests/queries/0_stateless/02033_join_engine_deadlock.reference
new file mode 100644
index 00000000000..e69de29bb2d
diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.sh b/tests/queries/0_stateless/02033_join_engine_deadlock.sh
new file mode 100755
index 00000000000..7a4ca1c8bb1
--- /dev/null
+++ b/tests/queries/0_stateless/02033_join_engine_deadlock.sh
@@ -0,0 +1,71 @@
+#!/usr/bin/env bash
+# Tags: deadlock
+
+CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
+# shellcheck source=../shell_config.sh
+. "$CURDIR"/../shell_config.sh
+
+create_table () {
+ $CLICKHOUSE_CLIENT --query "
+ CREATE TABLE join_block_test
+ (
+ id String,
+ num Int64
+ )
+ ENGINE = Join(ANY, LEFT, id)
+ "
+}
+
+drop_table () {
+ # Force a sync drop to free the memory before ending the test
+ # Otherwise things get interesting if you run the test many times before the database is finally dropped
+ $CLICKHOUSE_CLIENT --query "
+ DROP TABLE join_block_test SYNC
+ "
+}
+
+populate_table_bg () {
+ (
+ $CLICKHOUSE_CLIENT --query "
+ INSERT INTO join_block_test
+ SELECT toString(number) as id, number * number as num
+ FROM system.numbers LIMIT 3000000
+ " --lock_acquire_timeout=20 >/dev/null
+ ) &
+}
+
+read_table_bg () {
+ (
+ $CLICKHOUSE_CLIENT --query "
+ SELECT *
+ FROM
+ (
+ SELECT toString(number) AS user_id
+ FROM system.numbers LIMIT 10000 OFFSET 20000
+ ) AS t1
+ LEFT JOIN
+ (
+ SELECT
+ *
+ FROM join_block_test AS i1
+ ANY LEFT JOIN
+ (
+ SELECT *
+ FROM join_block_test
+ ) AS i2 ON i1.id = toString(i2.num)
+ ) AS t2 ON t1.user_id = t2.id
+ " --lock_acquire_timeout=20 >/dev/null
+ ) &
+}
+
+create_table
+for _ in {1..5};
+do
+ populate_table_bg
+ sleep 0.05
+ read_table_bg
+ sleep 0.05
+done
+
+wait
+drop_table
From 0ee5c0bff570ec676a394e2e60dc934b1e640b53 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?=
Date: Wed, 29 Sep 2021 19:30:07 +0200
Subject: [PATCH 020/264] Use RWLock in StorageJoin to avoid deadlocks
---
src/Functions/FunctionJoinGet.cpp | 6 +--
src/Functions/FunctionJoinGet.h | 16 +++++---
src/Interpreters/ExpressionAnalyzer.cpp | 2 +-
src/Interpreters/HashJoin.cpp | 2 +-
src/Interpreters/HashJoin.h | 7 ++--
src/Storages/IStorage.h | 1 +
src/Storages/StorageJoin.cpp | 53 +++++++++++++++----------
src/Storages/StorageJoin.h | 13 +++---
src/Storages/StorageSet.cpp | 21 +++++-----
src/Storages/StorageSet.h | 8 ++--
10 files changed, 76 insertions(+), 53 deletions(-)
diff --git a/src/Functions/FunctionJoinGet.cpp b/src/Functions/FunctionJoinGet.cpp
index f0dff0ac7e4..df131538275 100644
--- a/src/Functions/FunctionJoinGet.cpp
+++ b/src/Functions/FunctionJoinGet.cpp
@@ -25,14 +25,14 @@ ColumnPtr ExecutableFunctionJoinGet::executeImpl(const ColumnsWithTypeA
auto key = arguments[i];
keys.emplace_back(std::move(key));
}
- return storage_join->joinGet(keys, result_columns).column;
+ return storage_join->joinGet(keys, result_columns, getContext()).column;
}
template
ExecutableFunctionPtr FunctionJoinGet::prepare(const ColumnsWithTypeAndName &) const
{
Block result_columns {{return_type->createColumn(), return_type, attr_name}};
- return std::make_unique>(table_lock, storage_join, result_columns);
+ return std::make_unique>(getContext(), table_lock, storage_join, result_columns);
}
static std::pair, String>
@@ -89,7 +89,7 @@ FunctionBasePtr JoinGetOverloadResolver::buildImpl(const ColumnsWithTyp
auto return_type = storage_join->joinGetCheckAndGetReturnType(data_types, attr_name, or_null);
auto table_lock = storage_join->lockForShare(getContext()->getInitialQueryId(), getContext()->getSettingsRef().lock_acquire_timeout);
- return std::make_unique>(table_lock, storage_join, attr_name, argument_types, return_type);
+ return std::make_unique>(getContext(), table_lock, storage_join, attr_name, argument_types, return_type);
}
void registerFunctionJoinGet(FunctionFactory & factory)
diff --git a/src/Functions/FunctionJoinGet.h b/src/Functions/FunctionJoinGet.h
index 3ddab51e2d9..2dd0cb9fdea 100644
--- a/src/Functions/FunctionJoinGet.h
+++ b/src/Functions/FunctionJoinGet.h
@@ -14,13 +14,15 @@ class StorageJoin;
using StorageJoinPtr = std::shared_ptr;
template
-class ExecutableFunctionJoinGet final : public IExecutableFunction
+class ExecutableFunctionJoinGet final : public IExecutableFunction, WithContext
{
public:
- ExecutableFunctionJoinGet(TableLockHolder table_lock_,
+ ExecutableFunctionJoinGet(ContextPtr context_,
+ TableLockHolder table_lock_,
StorageJoinPtr storage_join_,
const DB::Block & result_columns_)
- : table_lock(std::move(table_lock_))
+ : WithContext(context_)
+ , table_lock(std::move(table_lock_))
, storage_join(std::move(storage_join_))
, result_columns(result_columns_)
{}
@@ -42,15 +44,17 @@ private:
};
template
-class FunctionJoinGet final : public IFunctionBase
+class FunctionJoinGet final : public IFunctionBase, WithContext
{
public:
static constexpr auto name = or_null ? "joinGetOrNull" : "joinGet";
- FunctionJoinGet(TableLockHolder table_lock_,
+ FunctionJoinGet(ContextPtr context_,
+ TableLockHolder table_lock_,
StorageJoinPtr storage_join_, String attr_name_,
DataTypes argument_types_, DataTypePtr return_type_)
- : table_lock(std::move(table_lock_))
+ : WithContext(context_)
+ , table_lock(std::move(table_lock_))
, storage_join(storage_join_)
, attr_name(std::move(attr_name_))
, argument_types(std::move(argument_types_))
diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp
index 566ee60a3e6..89d7624f203 100644
--- a/src/Interpreters/ExpressionAnalyzer.cpp
+++ b/src/Interpreters/ExpressionAnalyzer.cpp
@@ -938,7 +938,7 @@ JoinPtr SelectQueryExpressionAnalyzer::makeTableJoin(
if (auto storage = analyzed_join->getStorageJoin())
{
std::tie(left_convert_actions, right_convert_actions) = analyzed_join->createConvertingActions(left_columns, {});
- return storage->getJoinLocked(analyzed_join);
+ return storage->getJoinLocked(analyzed_join, getContext());
}
joined_plan = buildJoinedPlan(getContext(), join_element, *analyzed_join, query_options);
diff --git a/src/Interpreters/HashJoin.cpp b/src/Interpreters/HashJoin.cpp
index 07872df8ce5..d88df9d3e30 100644
--- a/src/Interpreters/HashJoin.cpp
+++ b/src/Interpreters/HashJoin.cpp
@@ -744,7 +744,7 @@ bool HashJoin::addJoinedBlock(const Block & source_block, bool check_limits)
size_t total_rows = 0;
size_t total_bytes = 0;
{
- if (storage_join_lock.mutex())
+ if (storage_join_lock)
throw DB::Exception("addJoinedBlock called when HashJoin locked to prevent updates",
ErrorCodes::LOGICAL_ERROR);
diff --git a/src/Interpreters/HashJoin.h b/src/Interpreters/HashJoin.h
index 07fd6d5b89f..f1f1198e7d9 100644
--- a/src/Interpreters/HashJoin.h
+++ b/src/Interpreters/HashJoin.h
@@ -16,6 +16,7 @@
#include
#include
#include
+#include
#include
#include
@@ -334,9 +335,9 @@ public:
/// We keep correspondence between used_flags and hash table internal buffer.
/// Hash table cannot be modified during HashJoin lifetime and must be protected with lock.
- void setLock(std::shared_mutex & rwlock)
+ void setLock(RWLockImpl::LockHolder rwlock_holder)
{
- storage_join_lock = std::shared_lock(rwlock);
+ storage_join_lock = rwlock_holder;
}
void reuseJoinedData(const HashJoin & join);
@@ -391,7 +392,7 @@ private:
/// Should be set via setLock to protect hash table from modification from StorageJoin
/// If set HashJoin instance is not available for modification (addJoinedBlock)
- std::shared_lock storage_join_lock;
+ RWLockImpl::LockHolder storage_join_lock = nullptr;
void dataMapInit(MapsVariant &);
diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h
index 6ce17552ba1..2013cc5ecb6 100644
--- a/src/Storages/IStorage.h
+++ b/src/Storages/IStorage.h
@@ -219,6 +219,7 @@ private:
/// without locks.
MultiVersionStorageMetadataPtr metadata;
+protected:
RWLockImpl::LockHolder tryLockTimed(
const RWLock & rwlock, RWLockImpl::Type type, const String & query_id, const std::chrono::milliseconds & acquire_timeout) const;
diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp
index e45183591f2..e5574708de0 100644
--- a/src/Storages/StorageJoin.cpp
+++ b/src/Storages/StorageJoin.cpp
@@ -1,13 +1,13 @@
#include
#include
#include
+#include
#include
#include
#include
#include
#include
#include
-#include
#include
#include
#include
@@ -67,6 +67,14 @@ StorageJoin::StorageJoin(
restore();
}
+RWLockImpl::LockHolder StorageJoin::tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const
+{
+ const String query_id = ctx ? ctx->getInitialQueryId() : RWLockImpl::NO_QUERY;
+ const std::chrono::milliseconds acquire_timeout
+ = ctx ? ctx->getSettingsRef().lock_acquire_timeout : std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC);
+ return tryLockTimed(lock, type, query_id, acquire_timeout);
+}
+
SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
{
std::lock_guard mutate_lock(mutate_mutex);
@@ -74,10 +82,10 @@ SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataP
}
void StorageJoin::truncate(
- const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr, TableExclusiveLockHolder&)
+ const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx, TableExclusiveLockHolder&)
{
std::lock_guard mutate_lock(mutate_mutex);
- std::unique_lock lock(rwlock);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx);
disk->removeRecursive(path);
disk->createDirectories(path);
@@ -128,7 +136,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context)
}
/// Now acquire exclusive lock and modify storage.
- std::unique_lock lock(rwlock);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context);
join = std::move(new_data);
increment = 1;
@@ -152,7 +160,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context)
}
}
-HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join) const
+HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const
{
auto metadata_snapshot = getInMemoryMetadataPtr();
if (!analyzed_join->sameStrictnessAndKind(strictness, kind))
@@ -171,34 +179,36 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join)
analyzed_join->setRightKeys(key_names);
HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns());
- join_clone->setLock(rwlock);
+
+ RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx);
+ join_clone->setLock(holder);
join_clone->reuseJoinedData(*join);
return join_clone;
}
-void StorageJoin::insertBlock(const Block & block)
+void StorageJoin::insertBlock(const Block & block, ContextPtr ctx)
{
- std::unique_lock lock(rwlock);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx);
join->addJoinedBlock(block, true);
}
-size_t StorageJoin::getSize() const
+size_t StorageJoin::getSize(ContextPtr ctx) const
{
- std::shared_lock lock(rwlock);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx);
return join->getTotalRowCount();
}
-std::optional StorageJoin::totalRows(const Settings &) const
+std::optional StorageJoin::totalRows(const Settings &settings) const
{
- std::shared_lock lock(rwlock);
+ TableLockHolder holder = tryLockTimed(rwlock, RWLockImpl::Read, RWLockImpl::NO_QUERY, settings.lock_acquire_timeout);
return join->getTotalRowCount();
}
-std::optional StorageJoin::totalBytes(const Settings &) const
+std::optional StorageJoin::totalBytes(const Settings &settings) const
{
- std::shared_lock lock(rwlock);
+ TableLockHolder holder = tryLockTimed(rwlock, RWLockImpl::Read, RWLockImpl::NO_QUERY, settings.lock_acquire_timeout);
return join->getTotalByteCount();
}
@@ -207,9 +217,9 @@ DataTypePtr StorageJoin::joinGetCheckAndGetReturnType(const DataTypes & data_typ
return join->joinGetCheckAndGetReturnType(data_types, column_name, or_null);
}
-ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add) const
+ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr ctx) const
{
- std::shared_lock lock(rwlock);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx);
return join->joinGet(block, block_with_columns_to_add);
}
@@ -370,10 +380,10 @@ size_t rawSize(const StringRef & t)
class JoinSource : public SourceWithProgress
{
public:
- JoinSource(HashJoinPtr join_, std::shared_mutex & rwlock, UInt64 max_block_size_, Block sample_block_)
+ JoinSource(HashJoinPtr join_, TableLockHolder lock_holder_, UInt64 max_block_size_, Block sample_block_)
: SourceWithProgress(sample_block_)
, join(join_)
- , lock(rwlock)
+ , lock_holder(lock_holder_)
, max_block_size(max_block_size_)
, sample_block(std::move(sample_block_))
{
@@ -421,7 +431,7 @@ protected:
private:
HashJoinPtr join;
- std::shared_lock lock;
+ TableLockHolder lock_holder;
UInt64 max_block_size;
Block sample_block;
@@ -571,7 +581,7 @@ Pipe StorageJoin::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & /*query_info*/,
- ContextPtr /*context*/,
+ ContextPtr context,
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
unsigned /*num_streams*/)
@@ -579,7 +589,8 @@ Pipe StorageJoin::read(
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
Block source_sample_block = metadata_snapshot->getSampleBlockForColumns(column_names, getVirtuals(), getStorageID());
- return Pipe(std::make_shared(join, rwlock, max_block_size, source_sample_block));
+ RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context);
+ return Pipe(std::make_shared(join, std::move(holder), max_block_size, source_sample_block));
}
}
diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h
index 6a08773ecc8..4926194433c 100644
--- a/src/Storages/StorageJoin.h
+++ b/src/Storages/StorageJoin.h
@@ -2,7 +2,9 @@
#include
+#include
#include
+#include
#include
#include
@@ -35,7 +37,7 @@ public:
/// Return instance of HashJoin holding lock that protects from insertions to StorageJoin.
/// HashJoin relies on structure of hash table that's why we need to return it with locked mutex.
- HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join) const;
+ HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const;
/// Get result type for function "joinGet(OrNull)"
DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const;
@@ -43,7 +45,7 @@ public:
/// Execute function "joinGet(OrNull)" on data block.
/// Takes rwlock for read to prevent parallel StorageJoin updates during processing data block
/// (but not during processing whole query, it's safe for joinGet that doesn't involve `used_flags` from HashJoin)
- ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add) const;
+ ColumnWithTypeAndName joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr context) const;
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & metadata_snapshot, ContextPtr context) override;
@@ -73,12 +75,13 @@ private:
/// Protect state for concurrent use in insertFromBlock and joinBlock.
/// Lock is stored in HashJoin instance during query and blocks concurrent insertions.
- mutable std::shared_mutex rwlock;
+ mutable RWLock rwlock = RWLockImpl::create();
mutable std::mutex mutate_mutex;
- void insertBlock(const Block & block) override;
+ void insertBlock(const Block & block, ContextPtr ctx) override;
void finishInsert() override {}
- size_t getSize() const override;
+ size_t getSize(ContextPtr context) const override;
+ RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const;
protected:
StorageJoin(
diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp
index fe55123335a..c57dadf6d52 100644
--- a/src/Storages/StorageSet.cpp
+++ b/src/Storages/StorageSet.cpp
@@ -34,11 +34,11 @@ namespace ErrorCodes
}
-class SetOrJoinSink : public SinkToStorage
+class SetOrJoinSink : public SinkToStorage, WithContext
{
public:
SetOrJoinSink(
- StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_,
+ ContextPtr ctx, StorageSetOrJoinBase & table_, const StorageMetadataPtr & metadata_snapshot_,
const String & backup_path_, const String & backup_tmp_path_,
const String & backup_file_name_, bool persistent_);
@@ -60,6 +60,7 @@ private:
SetOrJoinSink::SetOrJoinSink(
+ ContextPtr ctx,
StorageSetOrJoinBase & table_,
const StorageMetadataPtr & metadata_snapshot_,
const String & backup_path_,
@@ -67,6 +68,7 @@ SetOrJoinSink::SetOrJoinSink(
const String & backup_file_name_,
bool persistent_)
: SinkToStorage(metadata_snapshot_->getSampleBlock())
+ , WithContext(ctx)
, table(table_)
, metadata_snapshot(metadata_snapshot_)
, backup_path(backup_path_)
@@ -84,7 +86,7 @@ void SetOrJoinSink::consume(Chunk chunk)
/// Sort columns in the block. This is necessary, since Set and Join count on the same column order in different blocks.
Block sorted_block = getHeader().cloneWithColumns(chunk.detachColumns()).sortColumns();
- table.insertBlock(sorted_block);
+ table.insertBlock(sorted_block, getContext());
if (persistent)
backup_stream.write(sorted_block);
}
@@ -104,10 +106,10 @@ void SetOrJoinSink::onFinish()
}
-SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr /*context*/)
+SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx)
{
UInt64 id = ++increment;
- return std::make_shared(*this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent);
+ return std::make_shared(ctx, *this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent);
}
@@ -155,10 +157,10 @@ StorageSet::StorageSet(
}
-void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block.getColumnsWithTypeAndName()); }
+void StorageSet::insertBlock(const Block & block, ContextPtr) { set->insertFromBlock(block.getColumnsWithTypeAndName()); }
void StorageSet::finishInsert() { set->finishInsert(); }
-size_t StorageSet::getSize() const { return set->getTotalRowCount(); }
+size_t StorageSet::getSize(ContextPtr) const { return set->getTotalRowCount(); }
std::optional StorageSet::totalRows(const Settings &) const { return set->getTotalRowCount(); }
std::optional StorageSet::totalBytes(const Settings &) const { return set->getTotalByteCount(); }
@@ -210,6 +212,7 @@ void StorageSetOrJoinBase::restore()
void StorageSetOrJoinBase::restoreFromFile(const String & file_path)
{
+ ContextPtr ctx = nullptr;
auto backup_buf = disk->readFile(file_path);
CompressedReadBuffer compressed_backup_buf(*backup_buf);
NativeBlockInputStream backup_stream(compressed_backup_buf, 0);
@@ -217,14 +220,14 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path)
backup_stream.readPrefix();
while (Block block = backup_stream.read())
- insertBlock(block);
+ insertBlock(block, ctx);
finishInsert();
backup_stream.readSuffix();
/// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project.
LOG_INFO(&Poco::Logger::get("StorageSetOrJoinBase"), "Loaded from backup file {}. {} rows, {}. State has {} unique rows.",
- file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize());
+ file_path, backup_stream.getProfileInfo().rows, ReadableSize(backup_stream.getProfileInfo().bytes), getSize(ctx));
}
diff --git a/src/Storages/StorageSet.h b/src/Storages/StorageSet.h
index 1166557ec8e..1b78676b6c5 100644
--- a/src/Storages/StorageSet.h
+++ b/src/Storages/StorageSet.h
@@ -51,10 +51,10 @@ private:
void restoreFromFile(const String & file_path);
/// Insert the block into the state.
- virtual void insertBlock(const Block & block) = 0;
+ virtual void insertBlock(const Block & block, ContextPtr context) = 0;
/// Call after all blocks were inserted.
virtual void finishInsert() = 0;
- virtual size_t getSize() const = 0;
+ virtual size_t getSize(ContextPtr context) const = 0;
};
@@ -81,9 +81,9 @@ public:
private:
SetPtr set;
- void insertBlock(const Block & block) override;
+ void insertBlock(const Block & block, ContextPtr) override;
void finishInsert() override;
- size_t getSize() const override;
+ size_t getSize(ContextPtr) const override;
protected:
StorageSet(
From c0ba8d1a043c962c365c142e64a75a8e5db993b0 Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Wed, 29 Sep 2021 21:31:18 +0300
Subject: [PATCH 021/264] Fix crosslink.
---
docs/en/sql-reference/statements/create/view.md | 2 +-
docs/ru/sql-reference/statements/create/view.md | 2 +-
2 files changed, 2 insertions(+), 2 deletions(-)
diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md
index 84213020925..39c5760ecf3 100644
--- a/docs/en/sql-reference/statements/create/view.md
+++ b/docs/en/sql-reference/statements/create/view.md
@@ -50,7 +50,7 @@ When creating a materialized view with `TO [db].[table]`, you must not use `POPU
A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view.
!!! important "Important"
- Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views.
+ Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views.
Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view.
diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md
index 53d75b78dd1..9eb0baf5a98 100644
--- a/docs/ru/sql-reference/statements/create/view.md
+++ b/docs/ru/sql-reference/statements/create/view.md
@@ -49,7 +49,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na
!!! important "Важно"
- Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца.
+ Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца.
Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление.
From e53a48fb3089ab114b8fa907fc2efa3e27a1c960 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?=
Date: Thu, 30 Sep 2021 10:15:44 +0200
Subject: [PATCH 022/264] Raise lock acquire timeout for the test
Needed for check test under ASAN
---
tests/queries/0_stateless/02033_join_engine_deadlock.sh | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/tests/queries/0_stateless/02033_join_engine_deadlock.sh b/tests/queries/0_stateless/02033_join_engine_deadlock.sh
index 7a4ca1c8bb1..f4ae564e2a7 100755
--- a/tests/queries/0_stateless/02033_join_engine_deadlock.sh
+++ b/tests/queries/0_stateless/02033_join_engine_deadlock.sh
@@ -30,7 +30,7 @@ populate_table_bg () {
INSERT INTO join_block_test
SELECT toString(number) as id, number * number as num
FROM system.numbers LIMIT 3000000
- " --lock_acquire_timeout=20 >/dev/null
+ " >/dev/null
) &
}
@@ -54,7 +54,7 @@ read_table_bg () {
FROM join_block_test
) AS i2 ON i1.id = toString(i2.num)
) AS t2 ON t1.user_id = t2.id
- " --lock_acquire_timeout=20 >/dev/null
+ " >/dev/null
) &
}
From f58742014c0a72e9d02d9c02d4172b3d36989461 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?=
Date: Thu, 30 Sep 2021 10:47:15 +0200
Subject: [PATCH 023/264] Consistent naming
---
src/Storages/StorageJoin.cpp | 27 +++++++++++++--------------
src/Storages/StorageJoin.h | 6 +++---
2 files changed, 16 insertions(+), 17 deletions(-)
diff --git a/src/Storages/StorageJoin.cpp b/src/Storages/StorageJoin.cpp
index e5574708de0..b17315106ce 100644
--- a/src/Storages/StorageJoin.cpp
+++ b/src/Storages/StorageJoin.cpp
@@ -67,11 +67,11 @@ StorageJoin::StorageJoin(
restore();
}
-RWLockImpl::LockHolder StorageJoin::tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const
+RWLockImpl::LockHolder StorageJoin::tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr context) const
{
- const String query_id = ctx ? ctx->getInitialQueryId() : RWLockImpl::NO_QUERY;
+ const String query_id = context ? context->getInitialQueryId() : RWLockImpl::NO_QUERY;
const std::chrono::milliseconds acquire_timeout
- = ctx ? ctx->getSettingsRef().lock_acquire_timeout : std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC);
+ = context ? context->getSettingsRef().lock_acquire_timeout : std::chrono::seconds(DBMS_DEFAULT_LOCK_ACQUIRE_TIMEOUT_SEC);
return tryLockTimed(lock, type, query_id, acquire_timeout);
}
@@ -81,11 +81,10 @@ SinkToStoragePtr StorageJoin::write(const ASTPtr & query, const StorageMetadataP
return StorageSetOrJoinBase::write(query, metadata_snapshot, context);
}
-void StorageJoin::truncate(
- const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx, TableExclusiveLockHolder&)
+void StorageJoin::truncate(const ASTPtr &, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, TableExclusiveLockHolder &)
{
std::lock_guard mutate_lock(mutate_mutex);
- TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context);
disk->removeRecursive(path);
disk->createDirectories(path);
@@ -160,7 +159,7 @@ void StorageJoin::mutate(const MutationCommands & commands, ContextPtr context)
}
}
-HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const
+HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context) const
{
auto metadata_snapshot = getInMemoryMetadataPtr();
if (!analyzed_join->sameStrictnessAndKind(strictness, kind))
@@ -180,7 +179,7 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join,
HashJoinPtr join_clone = std::make_shared(analyzed_join, metadata_snapshot->getSampleBlock().sortColumns());
- RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx);
+ RWLockImpl::LockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context);
join_clone->setLock(holder);
join_clone->reuseJoinedData(*join);
@@ -188,15 +187,15 @@ HashJoinPtr StorageJoin::getJoinLocked(std::shared_ptr analyzed_join,
}
-void StorageJoin::insertBlock(const Block & block, ContextPtr ctx)
+void StorageJoin::insertBlock(const Block & block, ContextPtr context)
{
- TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, ctx);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Write, context);
join->addJoinedBlock(block, true);
}
-size_t StorageJoin::getSize(ContextPtr ctx) const
+size_t StorageJoin::getSize(ContextPtr context) const
{
- TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context);
return join->getTotalRowCount();
}
@@ -217,9 +216,9 @@ DataTypePtr StorageJoin::joinGetCheckAndGetReturnType(const DataTypes & data_typ
return join->joinGetCheckAndGetReturnType(data_types, column_name, or_null);
}
-ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr ctx) const
+ColumnWithTypeAndName StorageJoin::joinGet(const Block & block, const Block & block_with_columns_to_add, ContextPtr context) const
{
- TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, ctx);
+ TableLockHolder holder = tryLockTimedWithContext(rwlock, RWLockImpl::Read, context);
return join->joinGet(block, block_with_columns_to_add);
}
diff --git a/src/Storages/StorageJoin.h b/src/Storages/StorageJoin.h
index 4926194433c..cdc47531999 100644
--- a/src/Storages/StorageJoin.h
+++ b/src/Storages/StorageJoin.h
@@ -37,7 +37,7 @@ public:
/// Return instance of HashJoin holding lock that protects from insertions to StorageJoin.
/// HashJoin relies on structure of hash table that's why we need to return it with locked mutex.
- HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr ctx) const;
+ HashJoinPtr getJoinLocked(std::shared_ptr analyzed_join, ContextPtr context) const;
/// Get result type for function "joinGet(OrNull)"
DataTypePtr joinGetCheckAndGetReturnType(const DataTypes & data_types, const String & column_name, bool or_null) const;
@@ -78,10 +78,10 @@ private:
mutable RWLock rwlock = RWLockImpl::create();
mutable std::mutex mutate_mutex;
- void insertBlock(const Block & block, ContextPtr ctx) override;
+ void insertBlock(const Block & block, ContextPtr context) override;
void finishInsert() override {}
size_t getSize(ContextPtr context) const override;
- RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr ctx) const;
+ RWLockImpl::LockHolder tryLockTimedWithContext(const RWLock & lock, RWLockImpl::Type type, ContextPtr context) const;
protected:
StorageJoin(
From cfb9875acf3df38fe3f28188a0c79aff7a3e7a97 Mon Sep 17 00:00:00 2001
From: Artur <613623@mail.ru>
Date: Thu, 30 Sep 2021 13:47:12 +0000
Subject: [PATCH 024/264] Trigger Build
From 6f2447c027526d06e6f0125ea496815f01d40052 Mon Sep 17 00:00:00 2001
From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?=
Date: Thu, 30 Sep 2021 15:48:54 +0200
Subject: [PATCH 025/264] clang-tidy fix
---
src/Storages/StorageSet.cpp | 5 +++--
1 file changed, 3 insertions(+), 2 deletions(-)
diff --git a/src/Storages/StorageSet.cpp b/src/Storages/StorageSet.cpp
index c57dadf6d52..fd06c2975b6 100644
--- a/src/Storages/StorageSet.cpp
+++ b/src/Storages/StorageSet.cpp
@@ -106,10 +106,11 @@ void SetOrJoinSink::onFinish()
}
-SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr ctx)
+SinkToStoragePtr StorageSetOrJoinBase::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr context)
{
UInt64 id = ++increment;
- return std::make_shared(ctx, *this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent);
+ return std::make_shared(
+ context, *this, metadata_snapshot, path, fs::path(path) / "tmp/", toString(id) + ".bin", persistent);
}
From b2d13d0b199c7b648718247da310d33774e4f164 Mon Sep 17 00:00:00 2001
From: Vitaly Baranov
Date: Sun, 12 Sep 2021 16:43:22 +0300
Subject: [PATCH 026/264] Add test for expanding macros in RabbitMQ settings.
---
.../test_storage_rabbitmq/configs/macros.xml | 8 +++++
.../integration/test_storage_rabbitmq/test.py | 33 ++++++++++++++++++-
2 files changed, 40 insertions(+), 1 deletion(-)
create mode 100644 tests/integration/test_storage_rabbitmq/configs/macros.xml
diff --git a/tests/integration/test_storage_rabbitmq/configs/macros.xml b/tests/integration/test_storage_rabbitmq/configs/macros.xml
new file mode 100644
index 00000000000..6e9f3390b39
--- /dev/null
+++ b/tests/integration/test_storage_rabbitmq/configs/macros.xml
@@ -0,0 +1,8 @@
+
+
+ rabbitmq1
+ 5672
+ macro
+ JSONEachRow
+
+
diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py
index 9e2752438f8..696294f4bde 100644
--- a/tests/integration/test_storage_rabbitmq/test.py
+++ b/tests/integration/test_storage_rabbitmq/test.py
@@ -18,7 +18,7 @@ from . import rabbitmq_pb2
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance',
- main_configs=['configs/rabbitmq.xml'],
+ main_configs=['configs/rabbitmq.xml', 'configs/macros.xml'],
with_rabbitmq=True)
@@ -233,6 +233,37 @@ def test_rabbitmq_tsv_with_delimiter(rabbitmq_cluster):
rabbitmq_check_result(result, True)
+def test_rabbitmq_macros(rabbitmq_cluster):
+ instance.query('''
+ CREATE TABLE test.rabbitmq (key UInt64, value UInt64)
+ ENGINE = RabbitMQ
+ SETTINGS rabbitmq_host_port = '{rabbitmq_host}:{rabbitmq_port}',
+ rabbitmq_exchange_name = '{rabbitmq_exchange_name}',
+ rabbitmq_format = '{rabbitmq_format}'
+ ''')
+
+ credentials = pika.PlainCredentials('root', 'clickhouse')
+ parameters = pika.ConnectionParameters(rabbitmq_cluster.rabbitmq_ip, rabbitmq_cluster.rabbitmq_port, '/', credentials)
+ connection = pika.BlockingConnection(parameters)
+ channel = connection.channel()
+
+ message = ''
+ for i in range(50):
+ message += json.dumps({'key': i, 'value': i}) + '\n'
+ channel.basic_publish(exchange='macro', routing_key='', body=message)
+
+ connection.close()
+ time.sleep(1)
+
+ result = ''
+ while True:
+ result += instance.query('SELECT * FROM test.rabbitmq ORDER BY key', ignore_error=True)
+ if rabbitmq_check_result(result):
+ break
+
+ rabbitmq_check_result(result, True)
+
+
def test_rabbitmq_materialized_view(rabbitmq_cluster):
instance.query('''
DROP TABLE IF EXISTS test.view;
From ccdcaa21ff1696a1566374117caf49acece8358d Mon Sep 17 00:00:00 2001
From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com>
Date: Fri, 1 Oct 2021 16:13:10 +0300
Subject: [PATCH 027/264] Update install.sh
---
tests/config/install.sh | 1 +
1 file changed, 1 insertion(+)
diff --git a/tests/config/install.sh b/tests/config/install.sh
index f39bd951f46..764bc891c28 100755
--- a/tests/config/install.sh
+++ b/tests/config/install.sh
@@ -48,6 +48,7 @@ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/
+ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/
# FIXME DataPartsExchange may hang for http_send_timeout seconds
# when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"),
From 77081f33c29ebc436aec415d220c6a8ffb88e38f Mon Sep 17 00:00:00 2001
From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com>
Date: Fri, 1 Oct 2021 16:13:51 +0300
Subject: [PATCH 028/264] Update install.sh
---
tests/config/install.sh | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tests/config/install.sh b/tests/config/install.sh
index 764bc891c28..94ad55504a8 100755
--- a/tests/config/install.sh
+++ b/tests/config/install.sh
@@ -48,7 +48,6 @@ ln -sf $SRC_PATH/users.d/opentelemetry.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/remote_queries.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/session_log_test.xml $DEST_SERVER_PATH/users.d/
ln -sf $SRC_PATH/users.d/memory_profiler.xml $DEST_SERVER_PATH/users.d/
-ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/
# FIXME DataPartsExchange may hang for http_send_timeout seconds
# when nobody is going to read from the other side of socket (due to "Fetching of part was cancelled"),
@@ -60,6 +59,7 @@ ln -sf $SRC_PATH/strings_dictionary.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/decimals_dictionary.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/executable_dictionary.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/executable_pool_dictionary.xml $DEST_SERVER_PATH/
+ln -sf $SRC_PATH/test_function.xml $DEST_SERVER_PATH/
ln -sf $SRC_PATH/top_level_domains $DEST_SERVER_PATH/
From d41ef6a9f8b4ae47aadf8c572cd90c04208ad087 Mon Sep 17 00:00:00 2001
From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com>
Date: Fri, 1 Oct 2021 17:08:22 +0300
Subject: [PATCH 029/264] Update CORS.xml
---
tests/config/config.d/CORS.xml | 3 +--
1 file changed, 1 insertion(+), 2 deletions(-)
diff --git a/tests/config/config.d/CORS.xml b/tests/config/config.d/CORS.xml
index 873821478dc..b96209866a7 100644
--- a/tests/config/config.d/CORS.xml
+++ b/tests/config/config.d/CORS.xml
@@ -17,5 +17,4 @@
86400
-
-
+
From 50ef202b12aba727011ea2fe14f588f56e66a2d5 Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 1 Oct 2021 18:25:57 +0300
Subject: [PATCH 030/264] Update
docs/en/sql-reference/statements/create/view.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/en/sql-reference/statements/create/view.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md
index 39c5760ecf3..f174d561cc6 100644
--- a/docs/en/sql-reference/statements/create/view.md
+++ b/docs/en/sql-reference/statements/create/view.md
@@ -50,7 +50,7 @@ When creating a materialized view with `TO [db].[table]`, you must not use `POPU
A materialized view is implemented as follows: when inserting data to the table specified in `SELECT`, part of the inserted data is converted by this `SELECT` query, and the result is inserted in the view.
!!! important "Important"
- Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in `SELECT`'s result ClickHouse will use a default value, even if column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views.
+ Materialized views in ClickHouse use **column names** instead of column order during insertion into destination table. If some column names are not present in the `SELECT` query result, ClickHouse uses a default value, even if the column is not [Nullable](../../data-types/nullable.md). A safe practice would be to add aliases for every column when using Materialized views.
Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view.
From bb5c92276d0d9b5d838624b5cc345f33bbb41fdf Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 1 Oct 2021 18:26:09 +0300
Subject: [PATCH 031/264] Update
docs/en/sql-reference/statements/create/view.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/en/sql-reference/statements/create/view.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/en/sql-reference/statements/create/view.md b/docs/en/sql-reference/statements/create/view.md
index f174d561cc6..ec34c57a4cd 100644
--- a/docs/en/sql-reference/statements/create/view.md
+++ b/docs/en/sql-reference/statements/create/view.md
@@ -54,7 +54,7 @@ A materialized view is implemented as follows: when inserting data to the table
Materialized views in ClickHouse are implemented more like insert triggers. If there’s some aggregation in the view query, it’s applied only to the batch of freshly inserted data. Any changes to existing data of source table (like update, delete, drop partition, etc.) does not change the materialized view.
-If you specify `POPULATE`, the existing table data is inserted in the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it.
+If you specify `POPULATE`, the existing table data is inserted into the view when creating it, as if making a `CREATE TABLE ... AS SELECT ...` . Otherwise, the query contains only the data inserted in the table after creating the view. We **do not recommend** using `POPULATE`, since data inserted in the table during the view creation will not be inserted in it.
A `SELECT` query can contain `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`. Note that the corresponding conversions are performed independently on each block of inserted data. For example, if `GROUP BY` is set, data is aggregated during insertion, but only within a single packet of inserted data. The data won’t be further aggregated. The exception is when using an `ENGINE` that independently performs data aggregation, such as `SummingMergeTree`.
From fbe95f9c9d1834e09edd6c04b58ce58d09732f4e Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 1 Oct 2021 18:26:17 +0300
Subject: [PATCH 032/264] Update
docs/ru/sql-reference/functions/hash-functions.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/sql-reference/functions/hash-functions.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md
index 98b5ed6df27..975efdae71c 100644
--- a/docs/ru/sql-reference/functions/hash-functions.md
+++ b/docs/ru/sql-reference/functions/hash-functions.md
@@ -263,7 +263,7 @@ SHA512('s')
Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов).
Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать.
-Даже в этих случаях, рекомендуется применять функцию оффлайн - заранее вычисляя значения при вставке в таблицу, вместо того, чтобы применять её при выполнении `SELECT`.
+Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`.
**Параметры**
From 7d5ea307f1d51b434ca44072eaf51aba6e57e992 Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 1 Oct 2021 18:26:25 +0300
Subject: [PATCH 033/264] Update
docs/ru/sql-reference/functions/hash-functions.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/sql-reference/functions/hash-functions.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md
index 975efdae71c..18197f88ce3 100644
--- a/docs/ru/sql-reference/functions/hash-functions.md
+++ b/docs/ru/sql-reference/functions/hash-functions.md
@@ -262,7 +262,7 @@ SHA512('s')
```
Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов).
-Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хэш-функция и вы не можете её выбрать.
+Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хеш-функция и вы не можете её выбрать.
Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`.
**Параметры**
From 89f4830180ae0120797d75cc81e46ab5abd2ff2e Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 1 Oct 2021 18:26:33 +0300
Subject: [PATCH 034/264] Update
docs/ru/sql-reference/statements/create/view.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/sql-reference/statements/create/view.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md
index 9eb0baf5a98..77bdc7249c7 100644
--- a/docs/ru/sql-reference/statements/create/view.md
+++ b/docs/ru/sql-reference/statements/create/view.md
@@ -49,7 +49,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na
!!! important "Важно"
- Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse будет использовать значение по умолчанию, даже если столбец не является [Nullable](../../data-types/nullable.md). Безопасной практикой, при использовании материализованных представлений, считается добавление псевдонимов для каждого столбца.
+ Материализованные представления в ClickHouse используют **имена столбцов** вместо порядка следования столбцов при вставке в целевую таблицу. Если в результатах запроса `SELECT` некоторые имена столбцов отсутствуют, то ClickHouse использует значение по умолчанию, даже если столбец не является [Nullable](../../data-types/nullable.md). Безопасной практикой при использовании материализованных представлений считается добавление псевдонимов для каждого столбца.
Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление.
From 61a7db9612ed67ce4320bda3c193ec07669f4242 Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 1 Oct 2021 18:26:55 +0300
Subject: [PATCH 035/264] Update
docs/ru/sql-reference/statements/create/view.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/sql-reference/statements/create/view.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/statements/create/view.md b/docs/ru/sql-reference/statements/create/view.md
index 77bdc7249c7..7ebb154d6b6 100644
--- a/docs/ru/sql-reference/statements/create/view.md
+++ b/docs/ru/sql-reference/statements/create/view.md
@@ -53,7 +53,7 @@ CREATE MATERIALIZED VIEW [IF NOT EXISTS] [db.]table_name [ON CLUSTER] [TO[db.]na
Материализованные представления в ClickHouse больше похожи на `after insert` триггеры. Если в запросе материализованного представления есть агрегирование, оно применяется только к вставляемому блоку записей. Любые изменения существующих данных исходной таблицы (например обновление, удаление, удаление раздела и т.д.) не изменяют материализованное представление.
-Если указано `POPULATE`, то при создании представления, в него будут вставлены имеющиеся данные таблицы, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Иначе, представление будет содержать только данные, вставляемые в таблицу после создания представления. Не рекомендуется использовать `POPULATE`, так как вставляемые в таблицу данные во время создания представления, не попадут в него.
+Если указано `POPULATE`, то при создании представления в него будут добавлены данные, уже содержащиеся в исходной таблице, как если бы был сделан запрос `CREATE TABLE ... AS SELECT ...` . Если `POPULATE` не указано, представление будет содержать только данные, добавленные в таблицу после создания представления. Использовать `POPULATE` не рекомендуется, так как в представление не попадут данные, добавляемые в таблицу во время создания представления.
Запрос `SELECT` может содержать `DISTINCT`, `GROUP BY`, `ORDER BY`, `LIMIT`… Следует иметь ввиду, что соответствующие преобразования будут выполняться независимо, на каждый блок вставляемых данных. Например, при наличии `GROUP BY`, данные будут агрегироваться при вставке, но только в рамках одной пачки вставляемых данных. Далее, данные не будут доагрегированы. Исключение - использование ENGINE, производящего агрегацию данных самостоятельно, например, `SummingMergeTree`.
From a99a6fccc7289fb75bb55a3b47cfda8d144478b0 Mon Sep 17 00:00:00 2001
From: WangZengrui
Date: Sat, 2 Oct 2021 02:34:53 +0800
Subject: [PATCH 036/264] init
---
src/Interpreters/getOSKernelVersion.cpp | 37 +++++++++++++++++++++++++
src/Interpreters/getOSKernelVersion.h | 31 +++++++++++++++++++++
2 files changed, 68 insertions(+)
create mode 100644 src/Interpreters/getOSKernelVersion.cpp
create mode 100644 src/Interpreters/getOSKernelVersion.h
diff --git a/src/Interpreters/getOSKernelVersion.cpp b/src/Interpreters/getOSKernelVersion.cpp
new file mode 100644
index 00000000000..44df948be3c
--- /dev/null
+++ b/src/Interpreters/getOSKernelVersion.cpp
@@ -0,0 +1,37 @@
+#if defined(OS_LINUX)
+#include
+
+
+namespace DB
+{
+
+namespace ErrorCodes
+{
+ extern const int BAD_ARGUMENTS;
+}
+
+String getOSKernelVersion()
+{
+ struct utsname os_kernel_version;
+ int buf = uname(&os_kernel_version);
+ if (buf < 0)
+ {
+ throw Exception(
+ "EFAULT buf is not valid.",
+ ErrorCodes::BAD_ARGUMENTS);
+ }
+ else
+ {
+ // std::cout <<"sysname: " << os_kernel_version.sysname << " nodename: " << os_kernel_version.nodename
+ // << " release: " << os_kernel_version.release << " version: " << os_kernel_version.version
+ // << " machine: " << os_kernel_version.machine << std::endl;
+
+ return "sysname: " + String(os_kernel_version.sysname) + " nodename: " + String(os_kernel_version.nodename)
+ + " release: " + String(os_kernel_version.release) + " version: " + String(os_kernel_version.version)
+ + " machine: " + String(os_kernel_version.machine);
+ }
+}
+
+}
+
+#endif
\ No newline at end of file
diff --git a/src/Interpreters/getOSKernelVersion.h b/src/Interpreters/getOSKernelVersion.h
new file mode 100644
index 00000000000..14b42d2a19a
--- /dev/null
+++ b/src/Interpreters/getOSKernelVersion.h
@@ -0,0 +1,31 @@
+#if defined(OS_LINUX)
+#pragma once
+
+#include
+
+#include
+#include
+
+namespace DB
+{
+
+/// Returns String with OS Kernel version.
+/* To get name and information about current kernel.
+ For simplicity, the function can be implemented only for Linux.
+*/
+
+String getOSKernelVersion();
+
+// String getSysName();
+
+// String getNodeName();
+
+// String getReleaseName();
+
+// String getVersion();
+
+// String getMachineName();
+
+}
+
+#endif
\ No newline at end of file
From 271f7995c03f5e24a047e692fad06f81181c0d93 Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Mon, 4 Oct 2021 00:19:37 +0300
Subject: [PATCH 037/264] Fix PR comments.
---
.../sql-reference/functions/hash-functions.md | 130 ++---------------
.../sql-reference/functions/hash-functions.md | 132 ++----------------
2 files changed, 19 insertions(+), 243 deletions(-)
diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md
index e28594540be..20fe6d14e86 100644
--- a/docs/en/sql-reference/functions/hash-functions.md
+++ b/docs/en/sql-reference/functions/hash-functions.md
@@ -137,23 +137,29 @@ This is a relatively fast non-cryptographic hash function of average quality for
Calculates a 64-bit hash code from any type of integer.
It works faster than intHash32. Average quality.
-## SHA1 {#sha1}
+## SHA1, SHA224, SHA256, SHA512 {#sha}
-Calculates SHA-1 hash from a string and returns the resulting set of bytes as [FixedString(20)](../data-types/fixedstring.md).
+Calculates SHA-1, SHA-224, SHA-256, SHA-512 hash from a string and returns the resulting set of bytes as [FixedString](../data-types/fixedstring.md).
**Syntax**
``` sql
SHA1('s')
+...
+SHA512('s')
```
+The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million).
+We recommend using this function only in cases when you need a specific hash function and you can’t select it.
+Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in `SELECT` queries.
+
**Arguments**
- `s` — Input string for SHA-1 hash calculation. [String](../data-types/string.md).
**Returned value**
-- SHA-1 hash as a hex-unencoded FixedString(10).
+- SHA hash as a hex-unencoded FixedString. SHA-1 returns as FixedString(20), SHA-224 as FixedString(28), SHA-256 — FixedString(32), SHA-512 — FixedString(64).
Type: [FixedString](../data-types/fixedstring.md).
@@ -175,124 +181,6 @@ Result:
└──────────────────────────────────────────┘
```
-## SHA224 {#sha224}
-
-Calculates SHA-224 hash from a string and returns the resulting set of bytes as [FixedString(28)](../data-types/fixedstring.md).
-
-**Syntax**
-
-``` sql
-SHA224('s')
-```
-
-**Arguments**
-
-- `s` — Input string for SHA-224 hash calculation. [String](../data-types/string.md).
-
-**Returned value**
-
-- SHA-224 hash as a hex-unencoded FixedString(28).
-
-Type: [FixedString](../data-types/fixedstring.md).
-
-**Example**
-
-Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string.
-
-Query:
-
-``` sql
-SELECT hex(SHA224('abc'));
-```
-
-Result:
-
-``` text
-┌─hex(SHA224('abc'))───────────────────────────────────────┐
-│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │
-└──────────────────────────────────────────────────────────┘
-```
-
-## SHA256 {#sha256}
-
-Calculates SHA-256 hash from a string and returns the resulting set of bytes as [FixedString(32)](../data-types/fixedstring.md).
-
-**Syntax**
-
-``` sql
-SHA256('s')
-```
-
-**Arguments**
-
-- `s` — Input string for SHA-256 hash calculation. [String](../data-types/string.md).
-
-**Returned value**
-
-- SHA-256 hash as a hex-unencoded FixedString(32).
-
-Type: [FixedString](../data-types/fixedstring.md).
-
-**Example**
-
-Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string.
-
-Query:
-
-``` sql
-SELECT hex(SHA256('abc'));
-```
-
-Result:
-
-``` text
-┌─hex(SHA256('abc'))───────────────────────────────────────────────┐
-│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │
-└──────────────────────────────────────────────────────────────────┘
-```
-
-## SHA512 {#sha512}
-
-Calculates SHA-512 hash from a string and returns the resulting set of bytes as [FixedString(64)](../data-types/fixedstring.md).
-
-**Syntax**
-
-``` sql
-SHA512('s')
-```
-
-The function works fairly slowly (SHA-1 processes about 5 million short strings per second per processor core, while SHA-224 and SHA-256 process about 2.2 million).
-We recommend using this function only in cases when you need a specific hash function and you can’t select it.
-Even in these cases, we recommend applying the function offline and pre-calculating values when inserting them into the table, instead of applying it in `SELECT` queries.
-
-**Arguments**
-
-- `s` — Input string for SHA-512 hash calculation. [String](../data-types/string.md).
-
-**Returned value**
-
-- SHA-512 hash as a hex-unencoded FixedString(64).
-
-Type: [FixedString](../data-types/fixedstring.md).
-
-**Example**
-
-Use the [hex](../functions/encoding-functions.md#hex) function to represent the result as a hex-encoded string.
-
-Query:
-
-``` sql
-SELECT hex(SHA512('abc'));
-```
-
-Result:
-
-``` text
-┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
-│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │
-└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
-```
-
## URLHash(url\[, N\]) {#urlhashurl-n}
A fast, decent-quality non-cryptographic hash function for a string obtained from a URL using some type of normalization.
diff --git a/docs/ru/sql-reference/functions/hash-functions.md b/docs/ru/sql-reference/functions/hash-functions.md
index 18197f88ce3..f3b14625a8c 100644
--- a/docs/ru/sql-reference/functions/hash-functions.md
+++ b/docs/ru/sql-reference/functions/hash-functions.md
@@ -137,23 +137,29 @@ SELECT groupBitXor(cityHash64(*)) FROM table
Вычисляет 64-битный хэш-код от целого числа любого типа.
Работает быстрее, чем intHash32. Качество среднее.
-## SHA1 {#sha1}
+## SHA1, SHA224, SHA256, SHA512 {#sha}
-Вычисляет SHA-1 хеш строки и возвращает полученный набор байт в виде [FixedString(20)](../data-types/fixedstring.md).
+Вычисляет SHA-1, SHA-224, SHA-256, SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString](../data-types/fixedstring.md).
**Синтаксис**
``` sql
SHA1('s')
+...
+SHA512('s')
```
+Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов).
+Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хеш-функция и вы не можете её выбрать.
+Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`.
+
**Параметры**
-- `s` — входная строка для вычисления хеша SHA-1. [String](../data-types/string.md).
+- `s` — входная строка для вычисления хеша SHA. [String](../data-types/string.md).
**Возвращаемое значение**
-- Хеш SHA-1 в виде шестнадцатеричной некодированной строки FixedString(20).
+- Хеш SHA в виде шестнадцатеричной некодированной строки FixedString. SHA-1 хеш как FixedString(20), SHA-224 как FixedString(28), SHA-256 — FixedString(32), SHA-512 — FixedString(64).
Тип: [FixedString](../data-types/fixedstring.md).
@@ -175,124 +181,6 @@ SELECT hex(SHA1('abc'));
└──────────────────────────────────────────┘
```
-## SHA224 {#sha224}
-
-Вычисляет SHA-224 хеш строки и возвращает полученный набор байт в виде [FixedString(28)](../data-types/fixedstring.md).
-
-**Синтаксис**
-
-``` sql
-SHA224('s')
-```
-
-**Параметры**
-
-- `s` — входная строка для вычисления хеша SHA-224. [String](../data-types/string.md).
-
-**Возвращаемое значение**
-
-- Хеш SHA-224 в виде шестнадцатеричной некодированной строки FixedString(28).
-
-Тип: [FixedString](../data-types/fixedstring.md).
-
-**Пример**
-
-Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой.
-
-Запрос:
-
-``` sql
-SELECT hex(SHA224('abc'));
-```
-
-Результат:
-
-``` text
-┌─hex(SHA224('abc'))───────────────────────────────────────┐
-│ 23097D223405D8228642A477BDA255B32AADBCE4BDA0B3F7E36C9DA7 │
-└──────────────────────────────────────────────────────────┘
-```
-
-## SHA256 {#sha256}
-
-Вычисляет SHA-256 хеш строки и возвращает полученный набор байт в виде [FixedString(32)](../data-types/fixedstring.md).
-
-**Синтаксис**
-
-``` sql
-SHA256('s')
-```
-
-**Параметры**
-
-- `s` — входная строка для вычисления хеша SHA-256. [String](../data-types/string.md).
-
-**Возвращаемое значение**
-
-- Хеш SHA-256 в виде шестнадцатеричной некодированной строки FixedString(32).
-
-Тип: [FixedString](../data-types/fixedstring.md).
-
-**Пример**
-
-Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой.
-
-Запрос:
-
-``` sql
-SELECT hex(SHA256('abc'));
-```
-
-Результат:
-
-``` text
-┌─hex(SHA256('abc'))───────────────────────────────────────────────┐
-│ BA7816BF8F01CFEA414140DE5DAE2223B00361A396177A9CB410FF61F20015AD │
-└──────────────────────────────────────────────────────────────────┘
-```
-
-## SHA512 {#sha512}
-
-Вычисляет SHA-512 хеш строки и возвращает полученный набор байт в виде [FixedString(64)](../data-types/fixedstring.md)
-
-**Синтаксис**
-
-``` sql
-SHA512('s')
-```
-
-Функция работает достаточно медленно (SHA-1 — примерно 5 миллионов коротких строк в секунду на одном процессорном ядре, SHA-224 и SHA-256 — примерно 2.2 миллионов).
-Рекомендуется использовать эти функции лишь в тех случаях, когда вам нужна конкретная хеш-функция и вы не можете её выбрать.
-Даже в этих случаях рекомендуется применять функцию офлайн — заранее вычисляя значения при вставке в таблицу, вместо того чтобы применять её при выполнении `SELECT`.
-
-**Параметры**
-
-- `s` — входная строка для вычисления хеша SHA-512. [String](../data-types/string.md).
-
-**Возвращаемое значение**
-
-- Хеш SHA-512 в виде шестнадцатеричной некодированной строки FixedString(64).
-
-Тип: [FixedString](../data-types/fixedstring.md).
-
-**Пример**
-
-Используйте функцию [hex](../functions/encoding-functions.md#hex) для представления результата в виде строки с шестнадцатеричной кодировкой.
-
-Запрос:
-
-``` sql
-SELECT hex(SHA512('abc'));
-```
-
-Результат:
-
-``` text
-┌─hex(SHA512('abc'))───────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
-│ DDAF35A193617ABACC417349AE20413112E6FA4E89A97EA20A9EEEE64B55D39A2192992A274FC1A836BA3C23A3FEEBBD454D4423643CE80E2A9AC94FA54CA49F │
-└──────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
-```
-
## URLHash(url\[, N\]) {#urlhashurl-n}
Быстрая не криптографическая хэш-функция неплохого качества для строки, полученной из URL путём некоторой нормализации.
From 57c5d9d3828b69935a71d6f472762bdbce93bb46 Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Mon, 4 Oct 2021 00:29:46 +0300
Subject: [PATCH 038/264] Update hash-functions.md
minor fix
---
docs/en/sql-reference/functions/hash-functions.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/en/sql-reference/functions/hash-functions.md b/docs/en/sql-reference/functions/hash-functions.md
index d8659b406df..21ed8d33098 100644
--- a/docs/en/sql-reference/functions/hash-functions.md
+++ b/docs/en/sql-reference/functions/hash-functions.md
@@ -159,7 +159,7 @@ Even in these cases, we recommend applying the function offline and pre-calculat
**Arguments**
-- `s` — Input string for SHA-1 hash calculation. [String](../data-types/string.md).
+- `s` — Input string for SHA hash calculation. [String](../data-types/string.md).
**Returned value**
From 4497f5094e518f0a9d16068c152c88b6bc4c5c98 Mon Sep 17 00:00:00 2001
From: Mikhail <71978106+michon470@users.noreply.github.com>
Date: Mon, 4 Oct 2021 16:19:51 +0300
Subject: [PATCH 039/264] Moved changes to this new branch
---
.../sql-reference/statements/alter/column.md | 36 ++++++++++++++++++-
1 file changed, 35 insertions(+), 1 deletion(-)
diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md
index 801690afbb2..8f9273c81ba 100644
--- a/docs/en/sql-reference/statements/alter/column.md
+++ b/docs/en/sql-reference/statements/alter/column.md
@@ -10,7 +10,7 @@ A set of queries that allow changing the table structure.
Syntax:
``` sql
-ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ...
+ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|MODIFY|MATERIALIZE COLUMN ...
```
In the query, specify a list of one or more comma-separated actions.
@@ -25,6 +25,7 @@ The following actions are supported:
- [COMMENT COLUMN](#alter_comment-column) — Adds a text comment to the column.
- [MODIFY COLUMN](#alter_modify-column) — Changes column’s type, default expression and TTL.
- [MODIFY COLUMN REMOVE](#modify-remove) — Removes one of the column properties.
+- [MATERIALIZE COLUMN](#materialize-column) — Materializes the column in the parts where the column is missing.
These actions are described in detail below.
@@ -193,6 +194,39 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL;
- [REMOVE TTL](ttl.md).
+## MATERIALIZE COLUMN {#materialize-column}
+
+Materializes the column in the parts where the column is missing. This is useful in case of creating a new column with complicated `DEFAULT` or `MATERIALIZED` expression. Calculation of the column directly on `SELECT` query can cause bigger request execution time, so it is reasonable to use `MATERIALIZE COLUMN` for such columns. To perform same manipulation for existing column, use `FINAL` modifier as shown below.
+
+Syntax:
+
+```sql
+ALTER TABLE table MATERIALIZE COLUMN col [FINAL];
+```
+
+**Example:**
+
+```sql
+DROP TABLE IF EXISTS tmp;
+SET mutations_sync = 2;
+CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple();
+INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20;
+ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x);
+SELECT groupArray(x), groupArray(s) FROM tmp;
+```
+
+**Result:**
+
+```sql
+┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐
+│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │
+└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘
+```
+
+**See Also**
+
+- [MATERIALIZED](../../statements/create/table.md#materialized).
+
## Limitations {#alter-query-limitations}
The `ALTER` query lets you create and delete separate elements (columns) in nested data structures, but not whole nested data structures. To add a nested data structure, you can add columns with a name like `name.nested_name` and the type `Array(T)`. A nested data structure is equivalent to multiple array columns with a name that has the same prefix before the dot.
From 52c5f2da7203eaaae8ea819bc8ef405dafacb1c2 Mon Sep 17 00:00:00 2001
From: Mikhail <71978106+michon470@users.noreply.github.com>
Date: Mon, 4 Oct 2021 16:41:50 +0300
Subject: [PATCH 040/264] =?UTF-8?q?=D0=9F=D0=B5=D1=80=D0=B5=D0=B2=D0=BE?=
=?UTF-8?q?=D0=B4=20+=20=D0=B4=D0=BE=D0=BF=D0=BE=D0=BB=D0=BD=D0=B5=D0=BD?=
=?UTF-8?q?=D0=B8=D0=B5=20=D0=B0=D0=BD=D0=B3=D0=BB=20=D0=B2=D0=B5=D1=80?=
=?UTF-8?q?=D1=81=D0=B8=D0=B8?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
---
.../sql-reference/statements/alter/column.md | 2 +-
.../sql-reference/statements/alter/column.md | 38 +++++++++++++++++--
2 files changed, 35 insertions(+), 5 deletions(-)
diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md
index 8f9273c81ba..31874ef208d 100644
--- a/docs/en/sql-reference/statements/alter/column.md
+++ b/docs/en/sql-reference/statements/alter/column.md
@@ -204,7 +204,7 @@ Syntax:
ALTER TABLE table MATERIALIZE COLUMN col [FINAL];
```
-**Example:**
+**Example with the creation of new column:**
```sql
DROP TABLE IF EXISTS tmp;
diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md
index 9f59c79bfdd..366caf6a2a0 100644
--- a/docs/ru/sql-reference/statements/alter/column.md
+++ b/docs/ru/sql-reference/statements/alter/column.md
@@ -10,7 +10,7 @@ toc_title: "Манипуляции со столбцами"
Синтаксис:
``` sql
-ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN ...
+ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|MODIFY|MATERIALIZE COLUMN ...
```
В запросе можно указать сразу несколько действий над одной таблицей через запятую.
@@ -20,11 +20,12 @@ ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|CLEAR|COMMENT|MODIFY COLUMN
- [ADD COLUMN](#alter_add-column) — добавляет столбец в таблицу;
- [DROP COLUMN](#alter_drop-column) — удаляет столбец;
-- [RENAME COLUMN](#alter_rename-column) — переименовывает существующий столбец.
+- [RENAME COLUMN](#alter_rename-column) — переименовывает существующий столбец;
- [CLEAR COLUMN](#alter_clear-column) — сбрасывает все значения в столбце для заданной партиции;
- [COMMENT COLUMN](#alter_comment-column) — добавляет комментарий к столбцу;
-- [MODIFY COLUMN](#alter_modify-column) — изменяет тип столбца, выражение для значения по умолчанию и TTL.
-- [MODIFY COLUMN REMOVE](#modify-remove) — удаляет какое-либо из свойств столбца.
+- [MODIFY COLUMN](#alter_modify-column) — изменяет тип столбца, выражение для значения по умолчанию и TTL;
+- [MODIFY COLUMN REMOVE](#modify-remove) — удаляет какое-либо из свойств столбца;
+- [MATERIALIZE COLUMN](#materialize-column) — делает столбец материализованным (MATERIALIZED) в его частях, у которых отсутствуют значения.
Подробное описание для каждого действия приведено ниже.
@@ -193,6 +194,35 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL;
- [REMOVE TTL](ttl.md).
+## MATERIALIZE COLUMN {#materialize-column}
+
+С помощью этого запроса можно сделать столбец таблицы материализованным (`MATERIALIZED`) в его частях, у которых отсутствуют значения. Это полезно, если необходимо создать новый столбец со сложным материализованным выражением или выражением для заполнения по умолчанию (`DEFAULT`). Если вычисление такого столбца прямо во время выполнения запроса `SELECT` оказывается ощутимо большим, для него может оказаться целесообразным использовать `MATERIALIZE COLUMN`. Чтобы совершить ту же операцию для существующего столбца, используйте модификатор `FINAL`, как показано ниже.
+
+Синтаксис:
+
+```sql
+ALTER TABLE table MATERIALIZE COLUMN col [FINAL];
+```
+
+**Пример использования при создании нового столбца:**
+
+```sql
+DROP TABLE IF EXISTS tmp;
+SET mutations_sync = 2;
+CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple();
+INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20;
+ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x);
+SELECT groupArray(x), groupArray(s) FROM tmp;
+```
+
+**Результат:**
+
+```sql
+┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐
+│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │
+└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘
+```
+
## Ограничения запроса ALTER {#ogranicheniia-zaprosa-alter}
Запрос `ALTER` позволяет создавать и удалять отдельные элементы (столбцы) вложенных структур данных, но не вложенные структуры данных целиком. Для добавления вложенной структуры данных, вы можете добавить столбцы с именем вида `name.nested_name` и типом `Array(T)` - вложенная структура данных полностью эквивалентна нескольким столбцам-массивам с именем, имеющим одинаковый префикс до точки.
From a638c40fdcfd1b8b01153713e070664c1c38976f Mon Sep 17 00:00:00 2001
From: WangZengrui
Date: Tue, 5 Oct 2021 09:08:16 +0800
Subject: [PATCH 041/264] add getOSKernelVersion
---
src/Interpreters/getOSKernelVersion.cpp | 18 ++++++------------
src/Interpreters/getOSKernelVersion.h | 12 +-----------
2 files changed, 7 insertions(+), 23 deletions(-)
diff --git a/src/Interpreters/getOSKernelVersion.cpp b/src/Interpreters/getOSKernelVersion.cpp
index 44df948be3c..c4b4564f46e 100644
--- a/src/Interpreters/getOSKernelVersion.cpp
+++ b/src/Interpreters/getOSKernelVersion.cpp
@@ -7,28 +7,22 @@ namespace DB
namespace ErrorCodes
{
- extern const int BAD_ARGUMENTS;
+ extern const int SYSTEM_ERROR;
}
String getOSKernelVersion()
{
- struct utsname os_kernel_version;
- int buf = uname(&os_kernel_version);
+ struct utsname os_kernel_info;
+ int buf = uname(&os_kernel_info);
if (buf < 0)
{
throw Exception(
- "EFAULT buf is not valid.",
- ErrorCodes::BAD_ARGUMENTS);
+ "EFAULT buffer is not valid.",
+ ErrorCodes::SYSTEM_ERROR);
}
else
{
- // std::cout <<"sysname: " << os_kernel_version.sysname << " nodename: " << os_kernel_version.nodename
- // << " release: " << os_kernel_version.release << " version: " << os_kernel_version.version
- // << " machine: " << os_kernel_version.machine << std::endl;
-
- return "sysname: " + String(os_kernel_version.sysname) + " nodename: " + String(os_kernel_version.nodename)
- + " release: " + String(os_kernel_version.release) + " version: " + String(os_kernel_version.version)
- + " machine: " + String(os_kernel_version.machine);
+ return String(os_kernel_info.sysname) + " " + String(os_kernel_info.release);
}
}
diff --git a/src/Interpreters/getOSKernelVersion.h b/src/Interpreters/getOSKernelVersion.h
index 14b42d2a19a..fc3c7583aef 100644
--- a/src/Interpreters/getOSKernelVersion.h
+++ b/src/Interpreters/getOSKernelVersion.h
@@ -1,5 +1,5 @@
-#if defined(OS_LINUX)
#pragma once
+#if defined(OS_LINUX)
#include
@@ -16,16 +16,6 @@ namespace DB
String getOSKernelVersion();
-// String getSysName();
-
-// String getNodeName();
-
-// String getReleaseName();
-
-// String getVersion();
-
-// String getMachineName();
-
}
#endif
\ No newline at end of file
From 82e6ac8fa2cdba5ef016bbe4278d7c17888dafb7 Mon Sep 17 00:00:00 2001
From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com>
Date: Tue, 5 Oct 2021 13:39:18 +0300
Subject: [PATCH 042/264] Update HTTPHandler.cpp
---
src/Server/HTTPHandler.cpp | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index 413cfe18696..1036d5031f7 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -24,7 +24,7 @@
#include
#include
#include
-#include "common/logger_useful.h"
+#include
#include
#include
#include
From 9b1a39fdb9c4b79d2f045f88d97a1ce33c7d4797 Mon Sep 17 00:00:00 2001
From: Artur Filatenkov <613623@mail.ru>
Date: Tue, 5 Oct 2021 17:43:33 +0300
Subject: [PATCH 043/264] refactor after move common to base
---
src/Server/HTTPHandler.cpp | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/src/Server/HTTPHandler.cpp b/src/Server/HTTPHandler.cpp
index 1036d5031f7..9edef8a7223 100644
--- a/src/Server/HTTPHandler.cpp
+++ b/src/Server/HTTPHandler.cpp
@@ -24,15 +24,15 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
#include
#include
-#include
-#include
+#include
+#include
#include
#if !defined(ARCADIA_BUILD)
From fbfdd605eea214466bbd8d32a58f214aa5e5ca8e Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Tue, 5 Oct 2021 21:58:49 +0300
Subject: [PATCH 044/264] Update metrica.md
Update ru with for hits_100m_obfuscated
---
docs/ru/getting-started/example-datasets/metrica.md | 3 +++
1 file changed, 3 insertions(+)
diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md
index c82048a445e..27105ca8488 100644
--- a/docs/ru/getting-started/example-datasets/metrica.md
+++ b/docs/ru/getting-started/example-datasets/metrica.md
@@ -38,6 +38,9 @@ $ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=
$ # теперь создадим таблицу
$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
$ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
+# for hits_100m_obfuscated
+clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
+
$ # импортируем данные
$ cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000
$ # опционально можно оптимизировать таблицу
From af94e30a955dbfe271f412ee5ebe384994448f8e Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Tue, 5 Oct 2021 22:12:44 +0300
Subject: [PATCH 045/264] Update H3 functions
Update en and add ru draft.
---
docs/en/sql-reference/functions/geo/h3.md | 14 +--
docs/ru/sql-reference/functions/geo/h3.md | 112 +++++++++++++++++++++-
2 files changed, 118 insertions(+), 8 deletions(-)
diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md
index 3c3ed7b8932..9cdd3bcf947 100644
--- a/docs/en/sql-reference/functions/geo/h3.md
+++ b/docs/en/sql-reference/functions/geo/h3.md
@@ -481,7 +481,7 @@ Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT h3ToParent(599405990164561919, 3) as parent;
+SELECT h3ToParent(599405990164561919, 3) AS parent;
```
Result:
@@ -515,7 +515,7 @@ Type: [String](../../../sql-reference/data-types/string.md).
Query:
``` sql
-SELECT h3ToString(617420388352917503) as h3_string;
+SELECT h3ToString(617420388352917503) AS h3_string;
```
Result:
@@ -549,7 +549,7 @@ stringToH3(index_str)
Query:
``` sql
-SELECT stringToH3('89184926cc3ffff') as index;
+SELECT stringToH3('89184926cc3ffff') AS index;
```
Result:
@@ -583,7 +583,7 @@ h3GetResolution(index)
Query:
``` sql
-SELECT h3GetResolution(617420388352917503) as res;
+SELECT h3GetResolution(617420388352917503) AS res;
```
Result:
@@ -620,7 +620,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT h3IsResClassIII(617420388352917503) as res;
+SELECT h3IsResClassIII(617420388352917503) AS res;
```
Result:
@@ -657,7 +657,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT SELECT h3IsPentagon(644721767722457330) as pentagon;
+SELECT h3IsPentagon(644721767722457330) AS pentagon;
```
Result:
@@ -693,7 +693,7 @@ Type: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-
Query:
``` sql
-SELECT SELECT h3GetFaces(599686042433355775) as faces;
+SELECT h3GetFaces(599686042433355775) AS faces;
```
Result:
diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md
index bc47ca72a39..e8871d856c4 100644
--- a/docs/ru/sql-reference/functions/geo/h3.md
+++ b/docs/ru/sql-reference/functions/geo/h3.md
@@ -548,7 +548,7 @@ h3GetResolution(index)
Запрос:
``` sql
-SELECT h3GetResolution(617420388352917503) as res;
+SELECT h3GetResolution(617420388352917503) AS res;
```
Результат:
@@ -559,3 +559,113 @@ SELECT h3GetResolution(617420388352917503) as res;
└─────┘
```
+## h3IsResClassIII {#h3isresclassIII}
+
+Returns whether [H3](#h3index) index has a resolution with Class III orientation.
+
+**Синтаксис**
+
+``` sql
+h3IsResClassIII(index)
+```
+
+**Параметр**
+
+- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
+
+**Возвращаемые значения**
+
+- `1` — Index has a resolution with Class III orientation.
+- `0` — Index doesn't have a resolution with Class III orientation.
+
+Тип: [UInt8](../../../sql-reference/data-types/int-uint.md).
+
+**Пример**
+
+Запрос:
+
+``` sql
+SELECT h3IsResClassIII(617420388352917503) AS res;
+```
+
+Результат:
+
+``` text
+┌─res─┐
+│ 1 │
+└─────┘
+```
+
+## h3IsPentagon {#h3ispentagon }
+
+Returns whether this [H3](#h3index) index represents a pentagonal cell.
+
+**Синтаксис**
+
+``` sql
+h3IsPentagon(index)
+```
+
+**Параметр**
+
+- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
+
+**Возвращаемые значения**
+
+- `1` — Index represents a pentagonal cell.
+- `0` — Index doesn't represent a pentagonal cell.
+
+Тип: [UInt8](../../../sql-reference/data-types/int-uint.md).
+
+**Пример**
+
+Запрос:
+
+``` sql
+SELECT h3IsPentagon(644721767722457330) AS pentagon;
+```
+
+Результат:
+
+``` text
+┌─pentagon─┐
+│ 0 │
+└──────────┘
+```
+
+## h3GetFaces {#h3getfaces}
+
+Returns icosahedron faces intersected by a given [H3](#h3index) index.
+
+**Синтаксис**
+
+``` sql
+h3GetFaces(index)
+```
+
+**Параметр**
+
+- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
+
+**Возвращаемое значение**
+
+- Array containing icosahedron faces intersected by a given H3 index.
+
+Тип: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-reference/data-types/int-uint.md)).
+
+**Пример**
+
+Запрос:
+
+``` sql
+SELECT h3GetFaces(599686042433355775) AS faces;
+```
+
+Результат:
+
+``` text
+┌─faces─┐
+│ [7] │
+└───────┘
+
+[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3)
From bd4b0af2e14dfa22257d4778bf135f65dee1723c Mon Sep 17 00:00:00 2001
From: Alexey Milovidov
Date: Wed, 6 Oct 2021 02:34:48 +0300
Subject: [PATCH 046/264] Fix bad cast in ParserCreateQuery
---
src/Parsers/ParserCreateQuery.cpp | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp
index d4525883e36..2ea1663fc80 100644
--- a/src/Parsers/ParserCreateQuery.cpp
+++ b/src/Parsers/ParserCreateQuery.cpp
@@ -481,7 +481,7 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
if (attach && s_from.ignore(pos, expected))
{
- ParserLiteral from_path_p;
+ ParserStringLiteral from_path_p;
if (!from_path_p.parse(pos, from_path, expected))
return false;
}
From 1550c167bb8b725376968d2b1f2779c669f59a3a Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Wed, 6 Oct 2021 18:14:51 +0300
Subject: [PATCH 047/264] Update ru translation.
---
docs/en/sql-reference/functions/geo/h3.md | 2 +-
docs/ru/sql-reference/functions/geo/h3.md | 26 +++++++++++------------
2 files changed, 14 insertions(+), 14 deletions(-)
diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md
index 9cdd3bcf947..410cb9d3cc2 100644
--- a/docs/en/sql-reference/functions/geo/h3.md
+++ b/docs/en/sql-reference/functions/geo/h3.md
@@ -631,7 +631,7 @@ Result:
└─────┘
```
-## h3IsPentagon {#h3ispentagon }
+## h3IsPentagon {#h3ispentagon}
Returns whether this [H3](#h3index) index represents a pentagonal cell.
diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md
index e8871d856c4..cd807ade04a 100644
--- a/docs/ru/sql-reference/functions/geo/h3.md
+++ b/docs/ru/sql-reference/functions/geo/h3.md
@@ -6,7 +6,7 @@ toc_title: "Функции для работы с индексами H3"
[H3](https://eng.uber.com/h3/) — это система геокодирования, которая делит поверхность Земли на равные шестигранные ячейки. Система поддерживает иерархию (вложенность) ячеек, т.е. каждый "родительский" шестигранник может быть поделен на семь одинаковых вложенных "дочерних" шестигранников, и так далее.
-Уровень вложенности назвается `разрешением` и может принимать значение от `0` до `15`, где `0` соответствует `базовым` ячейкам самого верхнего уровня (наиболее крупным).
+Уровень вложенности называется `разрешением` и может принимать значение от `0` до `15`, где `0` соответствует `базовым` ячейкам самого верхнего уровня (наиболее крупным).
Для каждой точки, имеющей широту и долготу, можно получить 64-битный индекс H3, соответствующий номеру шестигранной ячейки, где эта точка находится.
@@ -561,7 +561,7 @@ SELECT h3GetResolution(617420388352917503) AS res;
## h3IsResClassIII {#h3isresclassIII}
-Returns whether [H3](#h3index) index has a resolution with Class III orientation.
+Проверяет, имеет ли индекс [H3](#h3index) разрешение с ориентацией Class III.
**Синтаксис**
@@ -571,12 +571,12 @@ h3IsResClassIII(index)
**Параметр**
-- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
+- `index` — порядковый номер шестигранника. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Возвращаемые значения**
-- `1` — Index has a resolution with Class III orientation.
-- `0` — Index doesn't have a resolution with Class III orientation.
+- `1` — индекс имеет разрешение с ориентацией Class III.
+- `0` — индекс не имеет разрешения с ориентацией Class III.
Тип: [UInt8](../../../sql-reference/data-types/int-uint.md).
@@ -596,9 +596,9 @@ SELECT h3IsResClassIII(617420388352917503) AS res;
└─────┘
```
-## h3IsPentagon {#h3ispentagon }
+## h3IsPentagon {#h3ispentagon}
-Returns whether this [H3](#h3index) index represents a pentagonal cell.
+Проверяет, является ли указанный индекс [H3](#h3index) пятиугольной ячейкой.
**Синтаксис**
@@ -608,12 +608,12 @@ h3IsPentagon(index)
**Параметр**
-- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
+- `index` — порядковый номер шестигранника. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Возвращаемые значения**
-- `1` — Index represents a pentagonal cell.
-- `0` — Index doesn't represent a pentagonal cell.
+- `1` — индекс представляет собой пятиугольную ячейку.
+- `0` — индекс не является пятиугольной ячейкой.
Тип: [UInt8](../../../sql-reference/data-types/int-uint.md).
@@ -635,7 +635,7 @@ SELECT h3IsPentagon(644721767722457330) AS pentagon;
## h3GetFaces {#h3getfaces}
-Returns icosahedron faces intersected by a given [H3](#h3index) index.
+Возвращает все грани многоугольника (икосаэдра), пересекаемые заданным [H3](#h3index) индексом.
**Синтаксис**
@@ -645,11 +645,11 @@ h3GetFaces(index)
**Параметр**
-- `index` — Hexagon index number. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
+- `index` — индекс шестиугольной ячейки. Тип: [UInt64](../../../sql-reference/data-types/int-uint.md).
**Возвращаемое значение**
-- Array containing icosahedron faces intersected by a given H3 index.
+- Массив, содержащий грани многоугольника (икосаэдра), пересекаемые заданным H3 индексом.
Тип: [Array](../../../sql-reference/data-types/array.md)([UInt64](../../../sql-reference/data-types/int-uint.md)).
From 259da1ccf07e8ed788ed7a418884c91801bff1fa Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Wed, 6 Oct 2021 18:32:55 +0300
Subject: [PATCH 048/264] Update h3.md
---
docs/ru/sql-reference/functions/geo/h3.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md
index cd807ade04a..6bc6943ec93 100644
--- a/docs/ru/sql-reference/functions/geo/h3.md
+++ b/docs/ru/sql-reference/functions/geo/h3.md
@@ -668,4 +668,4 @@ SELECT h3GetFaces(599686042433355775) AS faces;
│ [7] │
└───────┘
-[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3)
+[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3/)
From 4894588f2751189a55b0dce9ca218e4b0040ec7b Mon Sep 17 00:00:00 2001
From: Alexey
Date: Wed, 6 Oct 2021 19:50:05 +0000
Subject: [PATCH 049/264] description improved new example
---
.../functions/other-functions.md | 39 ++++++++++++++++---
1 file changed, 33 insertions(+), 6 deletions(-)
diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md
index 44702f4097f..2bb38684eb3 100644
--- a/docs/en/sql-reference/functions/other-functions.md
+++ b/docs/en/sql-reference/functions/other-functions.md
@@ -2357,8 +2357,8 @@ Result:
## shardNum {#shard-num}
-Returns the number of a shard which executes the query for a distributed query.
-If query is not distributed then *constant value* is returned.
+Returns the index of a shard which processes a part of data for a distributed query. Indices are started from `1`.
+If a query is not distributed then constant value `0` is returned.
**Syntax**
@@ -2368,14 +2368,39 @@ shardNum()
**Returned value**
-- Shard number.
+- Shard index or constant `0`.
Type: [UInt32](../../sql-reference/data-types/int-uint.md).
+**Example**
+
+In the following example a configuration with two shards is used. The query is executed on the [system.one](../../operations/system-tables/one.md) table on every shard.
+
+Query:
+
+``` sql
+CREATE TABLE shard_num_example (dummy UInt8)
+ ENGINE=Distributed(test_cluster_two_shards_localhost, system, one, dummy);
+SELECT dummy, shardNum(), shardCount() FROM shard_num_example;
+```
+
+Result:
+
+``` text
+┌─dummy─┬─shardNum()─┬─shardCount()─┐
+│ 0 │ 2 │ 2 │
+│ 0 │ 1 │ 2 │
+└───────┴────────────┴──────────────┘
+```
+
+**See Also**
+
+- [Distributed Table Engine](../../engines/table-engines/special/distributed.md)
+
## shardCount {#shard-count}
-Returns the total number of shards which execute a distributed query.
-If query is not distributed then *constant value* is returned.
+Returns the total number of shards for a distributed query.
+If a query is not distributed then constant value `0` is returned.
**Syntax**
@@ -2385,8 +2410,10 @@ shardCount()
**Returned value**
-- Total number of shards.
+- Total number of shards or `0`.
Type: [UInt32](../../sql-reference/data-types/int-uint.md).
+**See Also**
+- [shardNum()](#shard-num) function example also contains `shardCount()` function call.
From c2533b974394c2f1b4c356fa8408ad9dc0526d0f Mon Sep 17 00:00:00 2001
From: Alexey
Date: Wed, 6 Oct 2021 20:13:46 +0000
Subject: [PATCH 050/264] constant or column note added for other functions
---
docs/en/sql-reference/functions/date-time-functions.md | 1 +
docs/en/sql-reference/functions/other-functions.md | 4 ++++
2 files changed, 5 insertions(+)
diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md
index f54ef635e0c..b85f105758b 100644
--- a/docs/en/sql-reference/functions/date-time-functions.md
+++ b/docs/en/sql-reference/functions/date-time-functions.md
@@ -26,6 +26,7 @@ SELECT
## timeZone {#timezone}
Returns the timezone of the server.
+If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
**Syntax**
diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md
index 2bb38684eb3..afcc9563b58 100644
--- a/docs/en/sql-reference/functions/other-functions.md
+++ b/docs/en/sql-reference/functions/other-functions.md
@@ -8,6 +8,7 @@ toc_title: Other
## hostName() {#hostname}
Returns a string with the name of the host that this function was performed on. For distributed processing, this is the name of the remote server host, if the function is performed on a remote server.
+If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
## getMacro {#getmacro}
@@ -691,10 +692,12 @@ Returns the largest value of a and b.
## uptime() {#uptime}
Returns the server’s uptime in seconds.
+If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
## version() {#version}
Returns the version of the server as a string.
+If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
## blockNumber {#blocknumber}
@@ -2101,6 +2104,7 @@ UNSUPPORTED_METHOD
## tcpPort {#tcpPort}
Returns [native interface](../../interfaces/tcp.md) TCP port number listened by this server.
+If it is executed in the context of a distributed table, then it generates a normal column, otherwise it produces a constant value.
**Syntax**
From 95154305177f77d245d33493258e28e5df443fbc Mon Sep 17 00:00:00 2001
From: Azat Khuzhin
Date: Thu, 7 Oct 2021 00:17:14 +0300
Subject: [PATCH 051/264] Fix compilation with glibc 2.34 (MINSIGSTKSZ defined
as sysconf(_SC_SIGSTKSZ))
In glibc 2.34 MINSIGSTKSZ had been defined to sysconf(_SC_SIGSTKSZ) [1].
[1]: https://sourceware.org/git/?p=glibc.git;a=commit;h=6c57d320484988e87e446e2e60ce42816bf51d53
---
src/Common/ThreadStatus.cpp | 5 ++---
1 file changed, 2 insertions(+), 3 deletions(-)
diff --git a/src/Common/ThreadStatus.cpp b/src/Common/ThreadStatus.cpp
index b1d76c4660e..c0190cac58a 100644
--- a/src/Common/ThreadStatus.cpp
+++ b/src/Common/ThreadStatus.cpp
@@ -44,7 +44,7 @@ namespace
struct ThreadStack
{
ThreadStack()
- : data(aligned_alloc(getPageSize(), size))
+ : data(aligned_alloc(getPageSize(), getSize()))
{
/// Add a guard page
/// (and since the stack grows downward, we need to protect the first page).
@@ -56,12 +56,11 @@ struct ThreadStack
free(data);
}
- static size_t getSize() { return size; }
+ static size_t getSize() { return std::max(16 << 10, MINSIGSTKSZ); }
void * getData() const { return data; }
private:
/// 16 KiB - not too big but enough to handle error.
- static constexpr size_t size = std::max(16 << 10, MINSIGSTKSZ);
void * data;
};
From c41923c5958067f487b31a27f860cc1e775accdc Mon Sep 17 00:00:00 2001
From: Haavard Kvaalen
Date: Thu, 7 Oct 2021 13:29:38 +0200
Subject: [PATCH 052/264] MaterializedMySQL: Update GTID set at end of
transaction
We would update the set of seen GTIDs as soon as we saw a GTID_EVENT,
which arrives before a transaction. This would mostly work fine, but
if we lost the connection to MySQL in the middle of a large transaction
we would persist that the transaction had been processed as soon as the
transaction had started. When the connection was reestablished, we
would not process the transaction again, which meant that we only
applied parts of it.
Fix this by updating the seen GTIDs at the end of the transaction
instead.
---
src/Core/MySQL/MySQLReplication.cpp | 29 ++++++++++++++++--
src/Core/MySQL/MySQLReplication.h | 4 +++
.../materialize_with_ddl.py | 30 +++++++++++++++++++
.../test_materialized_mysql_database/test.py | 5 ++++
4 files changed, 65 insertions(+), 3 deletions(-)
diff --git a/src/Core/MySQL/MySQLReplication.cpp b/src/Core/MySQL/MySQLReplication.cpp
index 9c90b2ff220..b5468d15edc 100644
--- a/src/Core/MySQL/MySQLReplication.cpp
+++ b/src/Core/MySQL/MySQLReplication.cpp
@@ -105,12 +105,16 @@ namespace MySQLReplication
if (query.starts_with("BEGIN") || query.starts_with("COMMIT"))
{
typ = QUERY_EVENT_MULTI_TXN_FLAG;
+ if (!query.starts_with("COMMIT"))
+ transaction_complete = false;
}
else if (query.starts_with("XA"))
{
if (query.starts_with("XA ROLLBACK"))
throw ReplicationError("ParseQueryEvent: Unsupported query event:" + query, ErrorCodes::LOGICAL_ERROR);
typ = QUERY_EVENT_XA;
+ if (!query.starts_with("XA COMMIT"))
+ transaction_complete = false;
}
else if (query.starts_with("SAVEPOINT"))
{
@@ -711,9 +715,26 @@ namespace MySQLReplication
{
switch (event->header.type)
{
- case FORMAT_DESCRIPTION_EVENT:
- case QUERY_EVENT:
+ case FORMAT_DESCRIPTION_EVENT: {
+ binlog_pos = event->header.log_pos;
+ break;
+ }
+ case QUERY_EVENT: {
+ auto query = std::static_pointer_cast(event);
+ if (query->transaction_complete && pending_gtid)
+ {
+ gtid_sets.update(*pending_gtid);
+ pending_gtid.reset();
+ }
+ binlog_pos = event->header.log_pos;
+ break;
+ }
case XID_EVENT: {
+ if (pending_gtid)
+ {
+ gtid_sets.update(*pending_gtid);
+ pending_gtid.reset();
+ }
binlog_pos = event->header.log_pos;
break;
}
@@ -724,9 +745,11 @@ namespace MySQLReplication
break;
}
case GTID_EVENT: {
+ if (pending_gtid)
+ gtid_sets.update(*pending_gtid);
auto gtid_event = std::static_pointer_cast(event);
binlog_pos = event->header.log_pos;
- gtid_sets.update(gtid_event->gtid);
+ pending_gtid = gtid_event->gtid;
break;
}
default:
diff --git a/src/Core/MySQL/MySQLReplication.h b/src/Core/MySQL/MySQLReplication.h
index a57cc246eaa..cb67ce73de9 100644
--- a/src/Core/MySQL/MySQLReplication.h
+++ b/src/Core/MySQL/MySQLReplication.h
@@ -383,6 +383,7 @@ namespace MySQLReplication
String schema;
String query;
QueryType typ = QUERY_EVENT_DDL;
+ bool transaction_complete = true;
QueryEvent(EventHeader && header_)
: EventBase(std::move(header_)), thread_id(0), exec_time(0), schema_len(0), error_code(0), status_len(0)
@@ -536,6 +537,9 @@ namespace MySQLReplication
void update(BinlogEventPtr event);
void update(UInt64 binlog_pos_, const String & binlog_name_, const String & gtid_sets_);
void dump(WriteBuffer & out) const;
+
+ private:
+ std::optional pending_gtid;
};
class IFlavor : public MySQLProtocol::IMySQLReadPacket
diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py
index 23fa9894a84..5f6daea24ac 100644
--- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py
+++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py
@@ -980,3 +980,33 @@ def mysql_settings_test(clickhouse_node, mysql_node, service_name):
clickhouse_node.query("DROP DATABASE test_database")
mysql_node.query("DROP DATABASE test_database")
+def materialized_mysql_large_transaction(clickhouse_node, mysql_node, service_name):
+ mysql_node.query("DROP DATABASE IF EXISTS largetransaction")
+ clickhouse_node.query("DROP DATABASE IF EXISTS largetransaction")
+ mysql_node.query("CREATE DATABASE largetransaction")
+
+ mysql_node.query("CREATE TABLE largetransaction.test_table ("
+ "`key` INT NOT NULL PRIMARY KEY AUTO_INCREMENT, "
+ "`value` INT NOT NULL) ENGINE = InnoDB;")
+ num_rows = 200000
+ rows_per_insert = 5000
+ values = ",".join(["(1)" for _ in range(rows_per_insert)])
+ for i in range(num_rows//rows_per_insert):
+ mysql_node.query(f"INSERT INTO largetransaction.test_table (`value`) VALUES {values};")
+
+
+ clickhouse_node.query("CREATE DATABASE largetransaction ENGINE = MaterializedMySQL('{}:3306', 'largetransaction', 'root', 'clickhouse')".format(service_name))
+ check_query(clickhouse_node, "SELECT COUNT() FROM largetransaction.test_table", f"{num_rows}\n")
+
+ mysql_node.query("UPDATE largetransaction.test_table SET value = 2;")
+
+ # Attempt to restart clickhouse after it has started processing
+ # the transaction, but before it has completed it.
+ while int(clickhouse_node.query("SELECT COUNT() FROM largetransaction.test_table WHERE value = 2")) == 0:
+ time.sleep(0.2)
+ clickhouse_node.restart_clickhouse()
+
+ check_query(clickhouse_node, "SELECT COUNT() FROM largetransaction.test_table WHERE value = 2", f"{num_rows}\n")
+
+ clickhouse_node.query("DROP DATABASE largetransaction")
+ mysql_node.query("DROP DATABASE largetransaction")
diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py
index 18cb5b3b87c..feade1b60a0 100644
--- a/tests/integration/test_materialized_mysql_database/test.py
+++ b/tests/integration/test_materialized_mysql_database/test.py
@@ -237,3 +237,8 @@ def test_materialize_with_enum(started_cluster, started_mysql_8_0, started_mysql
def test_mysql_settings(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node):
materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_5_7, "mysql57")
materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_8_0, "mysql80")
+
+@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")])
+def test_large_transaction(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node):
+ materialize_with_ddl.materialized_mysql_large_transaction(clickhouse_node, started_mysql_8_0, "mysql80")
+ materialize_with_ddl.materialized_mysql_large_transaction(clickhouse_node, started_mysql_5_7, "mysql57")
From 7e2bc184ec29358cce749059bf776eccc784231e Mon Sep 17 00:00:00 2001
From: Alexander Tokmakov
Date: Thu, 7 Oct 2021 16:43:49 +0300
Subject: [PATCH 053/264] fix another suspicious places, add test
---
src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp | 2 +-
src/Parsers/ParserCreateQuery.cpp | 2 +-
src/Storages/MergeTree/MergeTreeIndexSet.cpp | 2 +-
tests/queries/0_stateless/01188_attach_table_from_path.sql | 1 +
4 files changed, 4 insertions(+), 3 deletions(-)
diff --git a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp
index f2860235117..a96713e3b5d 100644
--- a/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp
+++ b/src/Interpreters/MySQL/InterpretersMySQLDDLQuery.cpp
@@ -120,7 +120,7 @@ static NamesAndTypesList getColumnsList(const ASTExpressionList * columns_defini
auto * literal = child->as();
new_child->arguments = std::make_shared();
- new_child->arguments->children.push_back(std::make_shared(literal->value.get()));
+ new_child->arguments->children.push_back(std::make_shared(literal->value.safeGet()));
new_child->arguments->children.push_back(std::make_shared(Int16(++i)));
child = new_child;
}
diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp
index 2ea1663fc80..1da1bfba491 100644
--- a/src/Parsers/ParserCreateQuery.cpp
+++ b/src/Parsers/ParserCreateQuery.cpp
@@ -896,7 +896,7 @@ bool ParserCreateViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
if (ParserKeyword{"TO INNER UUID"}.ignore(pos, expected))
{
- ParserLiteral literal_p;
+ ParserStringLiteral literal_p;
if (!literal_p.parse(pos, to_inner_uuid, expected))
return false;
}
diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp
index 024b87c9a3e..60b9ddae329 100644
--- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp
+++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp
@@ -461,7 +461,7 @@ bool MergeTreeIndexConditionSet::checkASTUseless(const ASTPtr & node, bool atomi
[this](const auto & arg) { return checkASTUseless(arg, true); });
}
else if (const auto * literal = node->as())
- return !atomic && literal->value.get();
+ return !atomic && literal->value.safeGet();
else if (const auto * identifier = node->as())
return key_columns.find(identifier->getColumnName()) == std::end(key_columns);
else
diff --git a/tests/queries/0_stateless/01188_attach_table_from_path.sql b/tests/queries/0_stateless/01188_attach_table_from_path.sql
index 5b99c07e986..9bf401c8ea4 100644
--- a/tests/queries/0_stateless/01188_attach_table_from_path.sql
+++ b/tests/queries/0_stateless/01188_attach_table_from_path.sql
@@ -7,6 +7,7 @@ drop table if exists mt;
attach table test from 'some/path' (n UInt8) engine=Memory; -- { serverError 48 }
attach table test from '/etc/passwd' (s String) engine=File(TSVRaw); -- { serverError 481 }
attach table test from '../../../../../../../../../etc/passwd' (s String) engine=File(TSVRaw); -- { serverError 481 }
+attach table test from 42 (s String) engine=File(TSVRaw); -- { clientError 62 }
insert into table function file('01188_attach/file/data.TSV', 'TSV', 's String, n UInt8') values ('file', 42);
attach table file from '01188_attach/file' (s String, n UInt8) engine=File(TSV);
From d24bfce93fda2d35360213adc3f90936d8cab010 Mon Sep 17 00:00:00 2001
From: Nikolai Kochetov
Date: Thu, 7 Oct 2021 17:03:54 +0300
Subject: [PATCH 054/264] Add coroutines example.
---
src/Core/examples/CMakeLists.txt | 3 +
src/Core/examples/coro.cpp | 202 +++++++++++++++++++++++++++++++
2 files changed, 205 insertions(+)
create mode 100644 src/Core/examples/coro.cpp
diff --git a/src/Core/examples/CMakeLists.txt b/src/Core/examples/CMakeLists.txt
index 6b07dfbbfa6..c8846eb1743 100644
--- a/src/Core/examples/CMakeLists.txt
+++ b/src/Core/examples/CMakeLists.txt
@@ -13,3 +13,6 @@ target_link_libraries (mysql_protocol PRIVATE dbms)
if(USE_SSL)
target_include_directories (mysql_protocol SYSTEM PRIVATE ${OPENSSL_INCLUDE_DIR})
endif()
+
+add_executable (coro coro.cpp)
+target_link_libraries (coro PRIVATE clickhouse_common_io)
diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp
new file mode 100644
index 00000000000..c8e2f7418e4
--- /dev/null
+++ b/src/Core/examples/coro.cpp
@@ -0,0 +1,202 @@
+#include
+
+#include
+#include
+#include
+#include
+
+#include
+#include
+#include
+#include
+#include
+
+#if defined(__clang__)
+
+#include
+
+template
+using coroutine_handle = std::experimental::coroutine_handle;
+
+using default_coroutine_handle = std::experimental::coroutine_handle<>;
+
+using suspend_never = std::experimental::suspend_never;
+using suspend_always = std::experimental::suspend_always;
+
+#else
+
+#include
+
+template
+using coroutine_handle = std::coroutine_handle;
+
+using default_coroutine_handle = std::coroutine_handle<>;
+
+using suspend_never = std::suspend_never;
+using suspend_always = std::suspend_always;
+
+#endif
+
+
+template
+struct suspend_never_val
+{
+ constexpr bool await_ready() const noexcept { return true; }
+ constexpr void await_suspend(default_coroutine_handle) const noexcept {}
+ constexpr T await_resume() const noexcept
+ {
+ std::cout << " ret " << val << std::endl;
+ return val;
+ }
+
+ T val;
+};
+
+template
+struct resumable
+{
+ struct promise_type
+ {
+ using coro_handle = coroutine_handle;
+ auto get_return_object() { return coro_handle::from_promise(*this); }
+ auto initial_suspend() { return suspend_never(); }
+ auto final_suspend() noexcept { return suspend_never_val{*r->value}; }
+ //void return_void() {}
+ void return_value(T value_) { r->value = value_; }
+ void unhandled_exception()
+ {
+ DB::tryLogCurrentException("Logger");
+ r->exception = std::current_exception();
+ }
+
+ explicit promise_type(std::string tag_) : tag(tag_) {}
+ ~promise_type() { std::cout << "~promise_type " << tag << std::endl; }
+ std::string tag;
+ coro_handle next;
+ resumable * r = nullptr;
+ };
+
+ using coro_handle = coroutine_handle;
+
+ bool await_ready() const noexcept { return false; }
+ void await_suspend(coro_handle g) noexcept
+ {
+ std::cout << " await_suspend " << my.promise().tag << std::endl;
+ std::cout << " g tag " << g.promise().tag << std::endl;
+ g.promise().next = my;
+ }
+ T await_resume() noexcept
+ {
+ std::cout << " await_res " << my.promise().tag << std::endl;
+ return *value;
+ }
+
+ resumable(coro_handle handle) : my(handle), tag(handle.promise().tag)
+ {
+ assert(handle);
+ my.promise().r = this;
+ std::cout << " resumable " << tag << std::endl;
+ }
+ resumable(resumable &) = delete;
+ resumable(resumable &&rhs) : my(rhs.my), tag(rhs.tag)
+ {
+ rhs.my = {};
+ std::cout << " resumable&& " << tag << std::endl;
+ }
+ static bool resume_impl(resumable *r)
+ {
+ if (r->value)
+ return false;
+
+ auto & next = r->my.promise().next;
+
+ if (next)
+ {
+ if (resume_impl(next.promise().r))
+ return true;
+ next = {};
+ }
+
+ if (!r->value)
+ {
+ r->my.resume();
+ if (r->exception)
+ std::rethrow_exception(r->exception);
+ }
+ return !r->value;
+ }
+
+ bool resume()
+ {
+ return resume_impl(this);
+ }
+
+ T res()
+ {
+ return *value;
+ }
+
+ ~resumable()
+ {
+ std::cout << " ~resumable " << tag << std::endl;
+ }
+
+private:
+ coro_handle my;
+ std::string tag;
+ std::optional value;
+ std::exception_ptr exception;
+};
+
+resumable boo(std::string tag)
+{
+ std::cout << "x" << std::endl;
+ co_await suspend_always();
+ std::cout << StackTrace().toString();
+ std::cout << "y" << std::endl;
+ co_return 1;
+}
+
+resumable bar(std::string tag)
+{
+ std::cout << "a" << std::endl;
+ int res1 = co_await boo("boo1");
+ std::cout << "b " << res1 << std::endl;
+ int res2 = co_await boo("boo2");
+ if (res2 == 1)
+ throw DB::Exception(1, "hello");
+ std::cout << "c " << res2 << std::endl;
+ co_return res1 + res2; // 1 + 1 = 2
+}
+
+resumable foo(std::string tag) {
+ std::cout << "Hello" << std::endl;
+ auto res1 = co_await bar("bar1");
+ std::cout << "Coro " << res1 << std::endl;
+ auto res2 = co_await bar("bar2");
+ std::cout << "World " << res2 << std::endl;
+ co_return res1 * res2; // 2 * 2 = 4
+}
+
+int main()
+{
+ Poco::AutoPtr app_channel(new Poco::ConsoleChannel(std::cerr));
+ Poco::Logger::root().setChannel(app_channel);
+ Poco::Logger::root().setLevel("trace");
+
+ LOG_INFO(&Poco::Logger::get(""), "Starting");
+
+ try
+ {
+ auto t = foo("foo");
+ std::cout << ".. started" << std::endl;
+ while (t.resume())
+ std::cout << ".. yielded" << std::endl;
+ std::cout << ".. done: " << t.res() << std::endl;
+ }
+ catch (DB::Exception & e)
+ {
+ std::cout << "Got exception " << e.what() << std::endl;
+ std::cout << e.getStackTraceString() << std::endl;
+ }
+}
From 00fbf48a683ec009b26f0c47f931c7441013dbbe Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Thu, 7 Oct 2021 19:09:40 +0300
Subject: [PATCH 055/264] Minor fixes.
---
docs/en/sql-reference/functions/geo/h3.md | 14 +++++++-------
docs/ru/sql-reference/functions/geo/h3.md | 23 ++++++++++++-----------
2 files changed, 19 insertions(+), 18 deletions(-)
diff --git a/docs/en/sql-reference/functions/geo/h3.md b/docs/en/sql-reference/functions/geo/h3.md
index 410cb9d3cc2..048834806d1 100644
--- a/docs/en/sql-reference/functions/geo/h3.md
+++ b/docs/en/sql-reference/functions/geo/h3.md
@@ -40,7 +40,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT h3IsValid(630814730351855103) as h3IsValid;
+SELECT h3IsValid(630814730351855103) AS h3IsValid;
```
Result:
@@ -77,7 +77,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT h3GetResolution(639821929606596015) as resolution;
+SELECT h3GetResolution(639821929606596015) AS resolution;
```
Result:
@@ -111,7 +111,7 @@ h3EdgeAngle(resolution)
Query:
``` sql
-SELECT h3EdgeAngle(10) as edgeAngle;
+SELECT h3EdgeAngle(10) AS edgeAngle;
```
Result:
@@ -145,7 +145,7 @@ h3EdgeLengthM(resolution)
Query:
``` sql
-SELECT h3EdgeLengthM(15) as edgeLengthM;
+SELECT h3EdgeLengthM(15) AS edgeLengthM;
```
Result:
@@ -184,7 +184,7 @@ Type: [UInt64](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index;
+SELECT geoToH3(37.79506683, 55.71290588, 15) AS h3Index;
```
Result:
@@ -333,7 +333,7 @@ Type: [UInt8](../../../sql-reference/data-types/int-uint.md).
Query:
``` sql
-SELECT h3GetBaseCell(612916788725809151) as basecell;
+SELECT h3GetBaseCell(612916788725809151) AS basecell;
```
Result:
@@ -369,7 +369,7 @@ Type: [Float64](../../../sql-reference/data-types/float.md).
Query:
``` sql
-SELECT h3HexAreaM2(13) as area;
+SELECT h3HexAreaM2(13) AS area;
```
Result:
diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md
index 6bc6943ec93..e85236848f6 100644
--- a/docs/ru/sql-reference/functions/geo/h3.md
+++ b/docs/ru/sql-reference/functions/geo/h3.md
@@ -38,7 +38,7 @@ h3IsValid(h3index)
Запрос:
``` sql
-SELECT h3IsValid(630814730351855103) as h3IsValid;
+SELECT h3IsValid(630814730351855103) AS h3IsValid;
```
Результат:
@@ -75,7 +75,7 @@ h3GetResolution(h3index)
Запрос:
``` sql
-SELECT h3GetResolution(639821929606596015) as resolution;
+SELECT h3GetResolution(639821929606596015) AS resolution;
```
Результат:
@@ -109,7 +109,7 @@ h3EdgeAngle(resolution)
Запрос:
``` sql
-SELECT h3EdgeAngle(10) as edgeAngle;
+SELECT h3EdgeAngle(10) AS edgeAngle;
```
Результат:
@@ -143,7 +143,7 @@ h3EdgeLengthM(resolution)
Запрос:
``` sql
-SELECT h3EdgeLengthM(15) as edgeLengthM;
+SELECT h3EdgeLengthM(15) AS edgeLengthM;
```
Результат:
@@ -182,7 +182,7 @@ geoToH3(lon, lat, resolution)
Запрос:
``` sql
-SELECT geoToH3(37.79506683, 55.71290588, 15) as h3Index;
+SELECT geoToH3(37.79506683, 55.71290588, 15) AS h3Index;
```
Результат:
@@ -295,7 +295,7 @@ h3GetBaseCell(index)
Запрос:
``` sql
-SELECT h3GetBaseCell(612916788725809151) as basecell;
+SELECT h3GetBaseCell(612916788725809151) AS basecell;
```
Результат:
@@ -329,7 +329,7 @@ h3HexAreaM2(resolution)
Запрос:
``` sql
-SELECT h3HexAreaM2(13) as area;
+SELECT h3HexAreaM2(13) AS area;
```
Результат:
@@ -441,7 +441,7 @@ h3ToParent(index, resolution)
Запрос:
``` sql
-SELECT h3ToParent(599405990164561919, 3) as parent;
+SELECT h3ToParent(599405990164561919, 3) AS parent;
```
Результат:
@@ -475,7 +475,7 @@ h3ToString(index)
Запрос:
``` sql
-SELECT h3ToString(617420388352917503) as h3_string;
+SELECT h3ToString(617420388352917503) AS h3_string;
```
Результат:
@@ -512,7 +512,7 @@ stringToH3(index_str)
Запрос:
``` sql
-SELECT stringToH3('89184926cc3ffff') as index;
+SELECT stringToH3('89184926cc3ffff') AS index;
```
Результат:
@@ -667,5 +667,6 @@ SELECT h3GetFaces(599686042433355775) AS faces;
┌─faces─┐
│ [7] │
└───────┘
+```
-[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3/)
+[Оригинальная статья](https://clickhouse.com/docs/ru/sql-reference/functions/geo/h3)
From 1df9afb47cf5204be24edbe0e8c8c631ea1e759f Mon Sep 17 00:00:00 2001
From: michon470 <71978106+michon470@users.noreply.github.com>
Date: Thu, 7 Oct 2021 22:27:22 +0300
Subject: [PATCH 056/264] Update
docs/en/sql-reference/statements/alter/column.md
Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com>
---
docs/en/sql-reference/statements/alter/column.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md
index 31874ef208d..ef4b88af6ba 100644
--- a/docs/en/sql-reference/statements/alter/column.md
+++ b/docs/en/sql-reference/statements/alter/column.md
@@ -204,7 +204,7 @@ Syntax:
ALTER TABLE table MATERIALIZE COLUMN col [FINAL];
```
-**Example with the creation of new column:**
+**Example**
```sql
DROP TABLE IF EXISTS tmp;
From fc0bccb0c6f6fec55800235fde76ef6669c5b5f9 Mon Sep 17 00:00:00 2001
From: michon470 <71978106+michon470@users.noreply.github.com>
Date: Thu, 7 Oct 2021 22:27:39 +0300
Subject: [PATCH 057/264] Update
docs/ru/sql-reference/statements/alter/column.md
Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com>
---
docs/ru/sql-reference/statements/alter/column.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md
index 366caf6a2a0..5ab7207c580 100644
--- a/docs/ru/sql-reference/statements/alter/column.md
+++ b/docs/ru/sql-reference/statements/alter/column.md
@@ -196,7 +196,7 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL;
## MATERIALIZE COLUMN {#materialize-column}
-С помощью этого запроса можно сделать столбец таблицы материализованным (`MATERIALIZED`) в его частях, у которых отсутствуют значения. Это полезно, если необходимо создать новый столбец со сложным материализованным выражением или выражением для заполнения по умолчанию (`DEFAULT`). Если вычисление такого столбца прямо во время выполнения запроса `SELECT` оказывается ощутимо большим, для него может оказаться целесообразным использовать `MATERIALIZE COLUMN`. Чтобы совершить ту же операцию для существующего столбца, используйте модификатор `FINAL`, как показано ниже.
+Материализует столбец таблицы в кусках, в которых отсутствуют значения. Используется, если необходимо создать новый столбец со сложным материализованным выражением или выражением для заполнения по умолчанию (`DEFAULT`), потому как вычисление такого столбца прямо во время выполнения запроса `SELECT` оказывается ощутимо затратным. Чтобы совершить ту же операцию для существующего столбца, используйте модификатор `FINAL`.
Синтаксис:
From 0eaf2f12a31e1ed0f9dff5bfcd2059123541603f Mon Sep 17 00:00:00 2001
From: michon470 <71978106+michon470@users.noreply.github.com>
Date: Thu, 7 Oct 2021 22:27:46 +0300
Subject: [PATCH 058/264] Update
docs/ru/sql-reference/statements/alter/column.md
Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com>
---
docs/ru/sql-reference/statements/alter/column.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md
index 5ab7207c580..c6269f0eb62 100644
--- a/docs/ru/sql-reference/statements/alter/column.md
+++ b/docs/ru/sql-reference/statements/alter/column.md
@@ -204,7 +204,7 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL;
ALTER TABLE table MATERIALIZE COLUMN col [FINAL];
```
-**Пример использования при создании нового столбца:**
+**Пример**
```sql
DROP TABLE IF EXISTS tmp;
From c030756c38f75deaf0c3fd9e00c762e376d515c3 Mon Sep 17 00:00:00 2001
From: michon470 <71978106+michon470@users.noreply.github.com>
Date: Thu, 7 Oct 2021 22:28:02 +0300
Subject: [PATCH 059/264] Update
docs/en/sql-reference/statements/alter/column.md
Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com>
---
docs/en/sql-reference/statements/alter/column.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md
index ef4b88af6ba..aee3823bc05 100644
--- a/docs/en/sql-reference/statements/alter/column.md
+++ b/docs/en/sql-reference/statements/alter/column.md
@@ -196,7 +196,7 @@ ALTER TABLE table_with_ttl MODIFY COLUMN column_ttl REMOVE TTL;
## MATERIALIZE COLUMN {#materialize-column}
-Materializes the column in the parts where the column is missing. This is useful in case of creating a new column with complicated `DEFAULT` or `MATERIALIZED` expression. Calculation of the column directly on `SELECT` query can cause bigger request execution time, so it is reasonable to use `MATERIALIZE COLUMN` for such columns. To perform same manipulation for existing column, use `FINAL` modifier as shown below.
+Materializes the column in the parts where the column is missing. This is useful in case of creating a new column with complicated `DEFAULT` or `MATERIALIZED` expression. Calculation of the column directly on `SELECT` query can cause bigger request execution time, so it is reasonable to use `MATERIALIZE COLUMN` for such columns. To perform same manipulation for existing column, use `FINAL` modifier.
Syntax:
From 76e3ef686a244d13cbc37249ae260873ed36fae2 Mon Sep 17 00:00:00 2001
From: michon470 <71978106+michon470@users.noreply.github.com>
Date: Thu, 7 Oct 2021 22:28:12 +0300
Subject: [PATCH 060/264] Update
docs/ru/sql-reference/statements/alter/column.md
Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com>
---
docs/ru/sql-reference/statements/alter/column.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md
index c6269f0eb62..ef3d98fc10e 100644
--- a/docs/ru/sql-reference/statements/alter/column.md
+++ b/docs/ru/sql-reference/statements/alter/column.md
@@ -25,7 +25,7 @@ ALTER TABLE [db].name [ON CLUSTER cluster] ADD|DROP|RENAME|CLEAR|COMMENT|MODIFY|
- [COMMENT COLUMN](#alter_comment-column) — добавляет комментарий к столбцу;
- [MODIFY COLUMN](#alter_modify-column) — изменяет тип столбца, выражение для значения по умолчанию и TTL;
- [MODIFY COLUMN REMOVE](#modify-remove) — удаляет какое-либо из свойств столбца;
-- [MATERIALIZE COLUMN](#materialize-column) — делает столбец материализованным (MATERIALIZED) в его частях, у которых отсутствуют значения.
+- [MATERIALIZE COLUMN](#materialize-column) — делает столбец материализованным (`MATERIALIZED`) в кусках, в которых отсутствуют значения.
Подробное описание для каждого действия приведено ниже.
From 9389cb7c7702574dcf6224ef0e7c4d83e7a30896 Mon Sep 17 00:00:00 2001
From: Mikhail <71978106+michon470@users.noreply.github.com>
Date: Thu, 7 Oct 2021 22:31:44 +0300
Subject: [PATCH 061/264] Example corrected
---
docs/en/sql-reference/statements/alter/column.md | 8 ++++----
docs/ru/sql-reference/statements/alter/column.md | 8 ++++----
2 files changed, 8 insertions(+), 8 deletions(-)
diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md
index aee3823bc05..4eb251b88cd 100644
--- a/docs/en/sql-reference/statements/alter/column.md
+++ b/docs/en/sql-reference/statements/alter/column.md
@@ -210,7 +210,7 @@ ALTER TABLE table MATERIALIZE COLUMN col [FINAL];
DROP TABLE IF EXISTS tmp;
SET mutations_sync = 2;
CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple();
-INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20;
+INSERT INTO tmp SELECT * FROM system.numbers LIMIT 10;
ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x);
SELECT groupArray(x), groupArray(s) FROM tmp;
```
@@ -218,9 +218,9 @@ SELECT groupArray(x), groupArray(s) FROM tmp;
**Result:**
```sql
-┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐
-│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │
-└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘
+┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────┐
+│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','5','6','7','8','9'] │
+└───────────────────────┴───────────────────────────────────────────┘
```
**See Also**
diff --git a/docs/ru/sql-reference/statements/alter/column.md b/docs/ru/sql-reference/statements/alter/column.md
index ef3d98fc10e..bfd52801210 100644
--- a/docs/ru/sql-reference/statements/alter/column.md
+++ b/docs/ru/sql-reference/statements/alter/column.md
@@ -210,7 +210,7 @@ ALTER TABLE table MATERIALIZE COLUMN col [FINAL];
DROP TABLE IF EXISTS tmp;
SET mutations_sync = 2;
CREATE TABLE tmp (x Int64) ENGINE = MergeTree() ORDER BY tuple() PARTITION BY tuple();
-INSERT INTO tmp SELECT * FROM system.numbers LIMIT 20;
+INSERT INTO tmp SELECT * FROM system.numbers LIMIT 10;
ALTER TABLE tmp ADD COLUMN s String MATERIALIZED toString(x);
SELECT groupArray(x), groupArray(s) FROM tmp;
```
@@ -218,9 +218,9 @@ SELECT groupArray(x), groupArray(s) FROM tmp;
**Результат:**
```sql
-┌─groupArray(x)───────────────────────────────────────┬─groupArray(s)───────────────────────────────────────────────────────────────────────────────┐
-│ [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18,19] │ ['0','1','2','3','4','5','6','7','8','9','10','11','12','13','14','15','16','17','18','19'] │
-└─────────────────────────────────────────────────────┴─────────────────────────────────────────────────────────────────────────────────────────────┘
+┌─groupArray(x)─────────┬─groupArray(s)─────────────────────────────┐
+│ [0,1,2,3,4,5,6,7,8,9] │ ['0','1','2','3','4','5','6','7','8','9'] │
+└───────────────────────┴───────────────────────────────────────────┘
```
## Ограничения запроса ALTER {#ogranicheniia-zaprosa-alter}
From c734ada95ba7d5f13591acd32edb38f716784f64 Mon Sep 17 00:00:00 2001
From: kssenii
Date: Thu, 7 Oct 2021 20:26:58 +0000
Subject: [PATCH 062/264] Fix
---
src/Client/ClientBase.cpp | 4 +---
1 file changed, 1 insertion(+), 3 deletions(-)
diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp
index cde5a5f9977..b68df11fd60 100644
--- a/src/Client/ClientBase.cpp
+++ b/src/Client/ClientBase.cpp
@@ -426,10 +426,8 @@ void ClientBase::processTextAsSingleQuery(const String & full_query)
catch (Exception & e)
{
if (!is_interactive)
- {
e.addMessage("(in query: {})", full_query);
- throw;
- }
+ throw;
}
if (have_error)
From e07a6f3fc0ea0b496483287d85b50d29f5a8c330 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin
Date: Thu, 7 Oct 2021 21:09:35 +0300
Subject: [PATCH 063/264] docker: add pandas/clickhouse_driver into test images
---
docker/test/fasttest/Dockerfile | 2 +-
docker/test/fuzzer/Dockerfile | 2 +-
docker/test/stateless/Dockerfile | 2 +-
docker/test/style/Dockerfile | 2 +-
4 files changed, 4 insertions(+), 4 deletions(-)
diff --git a/docker/test/fasttest/Dockerfile b/docker/test/fasttest/Dockerfile
index 798910fb952..f50c65bb9f2 100644
--- a/docker/test/fasttest/Dockerfile
+++ b/docker/test/fasttest/Dockerfile
@@ -67,7 +67,7 @@ RUN apt-get update \
unixodbc \
--yes --no-install-recommends
-RUN pip3 install numpy scipy pandas Jinja2
+RUN pip3 install numpy scipy pandas Jinja2 pandas clickhouse_driver
# This symlink required by gcc to find lld compiler
RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld
diff --git a/docker/test/fuzzer/Dockerfile b/docker/test/fuzzer/Dockerfile
index 6444e745c47..13353bc2960 100644
--- a/docker/test/fuzzer/Dockerfile
+++ b/docker/test/fuzzer/Dockerfile
@@ -27,7 +27,7 @@ RUN apt-get update \
&& apt-get clean \
&& rm -rf /var/lib/apt/lists/*
-RUN pip3 install Jinja2
+RUN pip3 install Jinja2 pandas clickhouse_driver
COPY * /
diff --git a/docker/test/stateless/Dockerfile b/docker/test/stateless/Dockerfile
index 7de8c061673..a5733d11dd2 100644
--- a/docker/test/stateless/Dockerfile
+++ b/docker/test/stateless/Dockerfile
@@ -34,7 +34,7 @@ RUN apt-get update -y \
postgresql-client \
sqlite3
-RUN pip3 install numpy scipy pandas Jinja2
+RUN pip3 install numpy scipy pandas Jinja2 clickhouse_driver
RUN mkdir -p /tmp/clickhouse-odbc-tmp \
&& wget -nv -O - ${odbc_driver_url} | tar --strip-components=1 -xz -C /tmp/clickhouse-odbc-tmp \
diff --git a/docker/test/style/Dockerfile b/docker/test/style/Dockerfile
index 33cdb9db57a..64cc0c9c7b7 100644
--- a/docker/test/style/Dockerfile
+++ b/docker/test/style/Dockerfile
@@ -10,7 +10,7 @@ RUN apt-get update && env DEBIAN_FRONTEND=noninteractive apt-get install --yes \
python3-pip \
pylint \
yamllint \
- && pip3 install codespell
+ && pip3 install codespell pandas clickhouse_driver
COPY run.sh /
COPY process_style_check_result.py /
From 9dd0fca1edd383c00667ce4c1a953e4f6d2bca1a Mon Sep 17 00:00:00 2001
From: Azat Khuzhin
Date: Thu, 7 Oct 2021 23:45:18 +0300
Subject: [PATCH 064/264] Suppress some existed warnings in clickhouse-test
(will be fixed separately)
---
tests/clickhouse-test | 6 ++++++
1 file changed, 6 insertions(+)
diff --git a/tests/clickhouse-test b/tests/clickhouse-test
index 19080f3934f..2c8093190ea 100755
--- a/tests/clickhouse-test
+++ b/tests/clickhouse-test
@@ -1,6 +1,12 @@
#!/usr/bin/env python3
# pylint: disable=too-many-return-statements
+# pylint: disable=consider-using-f-string
+# pylint: disable=global-variable-not-assigned
+# pylint: disable=consider-using-with
+# pylint: disable=unspecified-encoding
+# pylint: disable=consider-using-min-builtin
+
import enum
import shutil
import sys
From df129d7efc70eb2abc394b72a0dd64c421de8549 Mon Sep 17 00:00:00 2001
From: Azat Khuzhin
Date: Thu, 7 Oct 2021 21:05:42 +0300
Subject: [PATCH 065/264] Rewrite clickhouse-test to use python
clickhouse_driver
Pros:
- Using native protocol over executing binaries is always better
- `clickhouse-client` in debug build takes almost a second to execute simple `SELECT 1`
and `clickhouse-test` requires ~5 queries at start (determine some
flags, zk, alive, create database)
Notes:
- `FORMAT Vertical` had been replaced with printing of `pandas.DataFrame`
And after this patch tiny tests work with the speed of the test, and
does not requires +-5 seconds of bootstrapping.
---
tests/clickhouse-test | 424 +++++++++++++++++++-----------------------
1 file changed, 193 insertions(+), 231 deletions(-)
diff --git a/tests/clickhouse-test b/tests/clickhouse-test
index 2c8093190ea..e8c85a6ae79 100755
--- a/tests/clickhouse-test
+++ b/tests/clickhouse-test
@@ -19,13 +19,10 @@ import traceback
import math
from argparse import ArgumentParser
-from typing import Tuple, Union, Optional, TextIO, Dict, Set, List
-import shlex
+from typing import Tuple, Union, Optional, Dict, Set, List
import subprocess
from subprocess import Popen
from subprocess import PIPE
-from subprocess import CalledProcessError
-from subprocess import TimeoutExpired
from datetime import datetime
from time import time, sleep
from errno import ESRCH
@@ -41,6 +38,9 @@ import multiprocessing
import socket
from contextlib import closing
+import clickhouse_driver
+import pandas
+
USE_JINJA = True
try:
import jinja2
@@ -48,20 +48,45 @@ except ImportError:
USE_JINJA = False
print('WARNING: jinja2 not installed! Template tests will be skipped.')
-DISTRIBUTED_DDL_TIMEOUT_MSG = "is executing longer than distributed_ddl_task_timeout"
-
MESSAGES_TO_RETRY = [
"ConnectionPoolWithFailover: Connection failed at try",
"DB::Exception: New table appeared in database being dropped or detached. Try again",
"is already started to be removing by another replica right now",
"DB::Exception: Cannot enqueue query",
- DISTRIBUTED_DDL_TIMEOUT_MSG # FIXME
+ "is executing longer than distributed_ddl_task_timeout" # FIXME
+]
+error_codes = clickhouse_driver.errors.ErrorCodes
+error_codes.NOT_A_LEADER = 529
+ERROR_CODES_TO_RETRY = [
+ error_codes.ALL_CONNECTION_TRIES_FAILED,
+ error_codes.DATABASE_NOT_EMPTY,
+ error_codes.NOT_A_LEADER,
+ error_codes.UNFINISHED,
]
MAX_RETRIES = 3
TEST_FILE_EXTENSIONS = ['.sql', '.sql.j2', '.sh', '.py', '.expect']
+class Client(clickhouse_driver.Client):
+ # return first column of the first row
+ def execute_one(self, *args, **kwargs):
+ return super().execute(*args, **kwargs)[0][0]
+
+ # return pandas.DataFrame
+ def execute_pandas(self, *args, **kwargs):
+ data = super().execute(*args, **kwargs, with_column_types=True)
+ return Client.__combine(data)
+
+ @staticmethod
+ def __combine(data):
+ cols = data[1]
+ rows = data[0]
+ header = [ i[0] for i in cols ]
+ data = pandas.DataFrame(data=rows, columns=header)
+ return data
+
+
class Terminated(KeyboardInterrupt):
pass
@@ -103,18 +128,16 @@ def get_db_engine(args, database_name):
def get_zookeeper_session_uptime(args):
+ global clickhouse_client
+
try:
- query = b"SELECT zookeeperSessionUptime()"
-
if args.replicated_database:
- query = b"SELECT min(materialize(zookeeperSessionUptime())) " \
- b"FROM clusterAllReplicas('test_cluster_database_replicated', system.one) "
-
- clickhouse_proc = open_client_process(args.client)
-
- (stdout, _) = clickhouse_proc.communicate((query), timeout=20)
-
- return int(stdout.decode('utf-8').strip())
+ return int(clickhouse_client.execute_one("""
+ SELECT min(materialize(zookeeperSessionUptime()))
+ FROM clusterAllReplicas('test_cluster_database_replicated', system.one)
+ """))
+ else:
+ return int(clickhouse_client.execute_one('SELECT zookeeperSessionUptime()'))
except:
return None
@@ -128,24 +151,31 @@ def need_retry(args, stdout, stderr, total_time):
return True
return any(msg in stdout for msg in MESSAGES_TO_RETRY) or any(msg in stderr for msg in MESSAGES_TO_RETRY)
+def need_retry_error(args, error, total_time):
+ # Sometimes we may get unexpected exception like "Replica is readonly" or "Shutdown is called for table"
+ # instead of "Session expired" or "Connection loss"
+ # Retry if session was expired during test execution
+ session_uptime = get_zookeeper_session_uptime(args)
+ if session_uptime is not None and session_uptime < math.ceil(total_time):
+ return True
+ if isinstance(error, clickhouse_driver.errors.Error):
+ if error.code in ERROR_CODES_TO_RETRY:
+ return True
+ if any(msg in error.message for msg in MESSAGES_TO_RETRY):
+ return True
+ return False
+
def get_processlist(args):
- try:
- query = b"SHOW PROCESSLIST FORMAT Vertical"
-
- if args.replicated_database:
- query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \
- b"FROM clusterAllReplicas('test_cluster_database_replicated', system.processes) " \
- b"WHERE query NOT LIKE '%system.processes%' FORMAT Vertical"
-
- clickhouse_proc = open_client_process(args.client)
-
- (stdout, _) = clickhouse_proc.communicate((query), timeout=20)
-
- return False, stdout.decode('utf-8')
- except Exception as ex:
- print("Exception", ex)
- return True, ""
+ global clickhouse_client
+ if args.replicated_database:
+ return clickhouse_client.execute_pandas("""
+ SELECT materialize((hostName(), tcpPort())) as host, *
+ FROM clusterAllReplicas('test_cluster_database_replicated', system.processes)
+ WHERE query NOT LIKE '%system.processes%'
+ """)
+ else:
+ return clickhouse_client.execute_pandas('SHOW PROCESSLIST')
# collect server stacktraces using gdb
@@ -311,7 +341,8 @@ class TestCase:
return None
@staticmethod
- def configure_testcase_args(args, case_file, suite_tmp_dir, stderr_file):
+ def configure_testcase_args(args, case_file, suite_tmp_dir):
+ global clickhouse_client
testcase_args = copy.deepcopy(args)
testcase_args.testcase_start_time = datetime.now()
@@ -331,23 +362,11 @@ class TestCase:
database = 'test_{suffix}'.format(suffix=random_str())
- with open(stderr_file, 'w') as stderr:
- client_cmd = testcase_args.testcase_client + " " \
- + get_additional_client_options(args)
-
- clickhouse_proc_create = open_client_process(
- universal_newlines=True,
- client_args=client_cmd,
- stderr_file=stderr)
-
- try:
- clickhouse_proc_create.communicate(
- ("CREATE DATABASE " + database + get_db_engine(testcase_args, database)),
- timeout=testcase_args.timeout)
- except TimeoutExpired:
- total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds()
- return clickhouse_proc_create, "", "Timeout creating database {} before test".format(
- database), total_time
+ try:
+ clickhouse_client.execute("CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename})
+ except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError):
+ total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds()
+ return None, "", f"Timeout creating database {database} before test", total_time
os.environ["CLICKHOUSE_DATABASE"] = database
# Set temporary directory to match the randomly generated database,
@@ -418,41 +437,42 @@ class TestCase:
def process_result_impl(self, proc, stdout: str, stderr: str, total_time: float):
description = ""
- if proc.returncode is None:
- try:
- proc.kill()
- except OSError as e:
- if e.errno != ESRCH:
- raise
+ if proc:
+ if proc.returncode is None:
+ try:
+ proc.kill()
+ except OSError as e:
+ if e.errno != ESRCH:
+ raise
- if stderr:
- description += stderr
- return TestResult(self.name, TestStatus.FAIL, FailureReason.TIMEOUT, total_time, description)
+ if stderr:
+ description += stderr
+ return TestResult(self.name, TestStatus.FAIL, FailureReason.TIMEOUT, total_time, description)
- if proc.returncode != 0:
- reason = FailureReason.EXIT_CODE
- description += str(proc.returncode)
+ if proc.returncode != 0:
+ reason = FailureReason.EXIT_CODE
+ description += str(proc.returncode)
- if stderr:
- description += "\n"
- description += stderr
+ if stderr:
+ description += "\n"
+ description += stderr
- # Stop on fatal errors like segmentation fault. They are sent to client via logs.
- if ' ' in stderr:
- reason = FailureReason.SERVER_DIED
+ # Stop on fatal errors like segmentation fault. They are sent to client via logs.
+ if ' ' in stderr:
+ reason = FailureReason.SERVER_DIED
- if self.testcase_args.stop \
- and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) \
- and 'Received exception from server' not in stderr:
- reason = FailureReason.SERVER_DIED
+ if self.testcase_args.stop \
+ and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) \
+ and 'Received exception from server' not in stderr:
+ reason = FailureReason.SERVER_DIED
- if os.path.isfile(self.stdout_file):
- description += ", result:\n\n"
- description += '\n'.join(open(self.stdout_file).read().split('\n')[:100])
- description += '\n'
+ if os.path.isfile(self.stdout_file):
+ description += ", result:\n\n"
+ description += '\n'.join(open(self.stdout_file).read().split('\n')[:100])
+ description += '\n'
- description += "\nstdout:\n{}\n".format(stdout)
- return TestResult(self.name, TestStatus.FAIL, reason, total_time, description)
+ description += "\nstdout:\n{}\n".format(stdout)
+ return TestResult(self.name, TestStatus.FAIL, reason, total_time, description)
if stderr:
description += "\n{}\n".format('\n'.join(stderr.split('\n')[:100]))
@@ -516,21 +536,12 @@ class TestCase:
@staticmethod
def send_test_name_failed(suite: str, case: str) -> bool:
- clickhouse_proc = open_client_process(args.client, universal_newlines=True)
-
- failed_to_check = False
-
+ global clickhouse_client
pid = os.getpid()
- query = f"SELECT 'Running test {suite}/{case} from pid={pid}';"
-
- try:
- clickhouse_proc.communicate((query), timeout=20)
- except:
- failed_to_check = True
-
- return failed_to_check or clickhouse_proc.returncode != 0
+ clickhouse_client.execute(f"SELECT 'Running test {suite}/{case} from pid={pid}'")
def run_single_test(self, server_logs_level, client_options):
+ global clickhouse_client
args = self.testcase_args
client = args.testcase_client
start_time = args.testcase_start_time
@@ -572,28 +583,13 @@ class TestCase:
need_drop_database = not maybe_passed
if need_drop_database:
- with open(self.stderr_file, 'a') as stderr:
- clickhouse_proc_create = open_client_process(client, universal_newlines=True, stderr_file=stderr)
-
seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20)
-
try:
- drop_database_query = "DROP DATABASE " + database
- if args.replicated_database:
- drop_database_query += " ON CLUSTER test_cluster_database_replicated"
- clickhouse_proc_create.communicate((drop_database_query), timeout=seconds_left)
- except TimeoutExpired:
- # kill test process because it can also hung
- if proc.returncode is None:
- try:
- proc.kill()
- except OSError as e:
- if e.errno != ESRCH:
- raise
-
+ with clickhouse_client.connection.timeout_setter(seconds_left):
+ clickhouse_client.execute("DROP DATABASE " + database)
+ except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError):
total_time = (datetime.now() - start_time).total_seconds()
- return clickhouse_proc_create, "", f"Timeout dropping database {database} after test", total_time
-
+ return None, "", f"Timeout dropping database {database} after test", total_time
shutil.rmtree(args.test_tmp_dir)
total_time = (datetime.now() - start_time).total_seconds()
@@ -624,12 +620,15 @@ class TestCase:
if skip_reason is not None:
return TestResult(self.name, TestStatus.SKIPPED, skip_reason, 0., "")
- if args.testname and self.send_test_name_failed(suite.suite, self.case):
- description = "\nServer does not respond to health check\n"
- return TestResult(self.name, TestStatus.FAIL, FailureReason.SERVER_DIED, 0., description)
+ if args.testname:
+ try:
+ self.send_test_name_failed(suite.suite, self.case)
+ except:
+ return TestResult(self.name, TestStatus.FAIL, FailureReason.SERVER_DIED, 0.,
+ "\nServer does not respond to health check\n")
self.runs_count += 1
- self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path, self.stderr_file)
+ self.testcase_args = self.configure_testcase_args(args, self.case_file, suite.suite_tmp_path)
proc, stdout, stderr, total_time = self.run_single_test(server_logs_level, client_options)
result = self.process_result_impl(proc, stdout, stderr, total_time)
@@ -794,12 +793,8 @@ class TestSuite:
@staticmethod
def readTestSuite(args, suite_dir_name: str):
def is_data_present():
- clickhouse_proc = open_client_process(args.client)
- (stdout, stderr) = clickhouse_proc.communicate(b"EXISTS TABLE test.hits")
- if clickhouse_proc.returncode != 0:
- raise CalledProcessError(clickhouse_proc.returncode, args.client, stderr)
-
- return stdout.startswith(b'1')
+ global clickhouse_client
+ return int(clickhouse_client.execute_one('EXISTS TABLE test.hits'))
base_dir = os.path.abspath(args.queries)
tmp_dir = os.path.abspath(args.tmp)
@@ -832,6 +827,7 @@ class TestSuite:
stop_time = None
+clickhouse_client = None
exit_code = None
server_died = None
stop_tests_triggered_lock = None
@@ -961,42 +957,26 @@ def run_tests_array(all_tests_with_params):
server_logs_level = "warning"
-def check_server_started(client, retry_count):
+def check_server_started(retry_count):
+ global clickhouse_client
print("Connecting to ClickHouse server...", end='')
sys.stdout.flush()
-
while retry_count > 0:
- clickhouse_proc = open_client_process(client)
- (stdout, stderr) = clickhouse_proc.communicate(b"SELECT 1")
-
- if clickhouse_proc.returncode == 0 and stdout.startswith(b"1"):
+ try:
+ clickhouse_client.execute('SELECT 1')
print(" OK")
sys.stdout.flush()
return True
-
- if clickhouse_proc.returncode == 210:
- # Connection refused, retry
+ except (ConnectionRefusedError, ConnectionResetError, clickhouse_driver.errors.NetworkError):
print('.', end='')
sys.stdout.flush()
retry_count -= 1
sleep(0.5)
continue
- code: int = clickhouse_proc.returncode
-
- print(f"\nClient invocation failed with code {code}:\n\
- stdout: {stdout}\n\
- stderr: {stderr}\n\
- args: {''.join(clickhouse_proc.args)}\n")
-
- sys.stdout.flush()
-
- return False
-
print('\nAll connection tries failed')
sys.stdout.flush()
-
return False
@@ -1012,60 +992,38 @@ class BuildFlags():
POLYMORPHIC_PARTS = 'polymorphic-parts'
-def collect_build_flags(client):
- clickhouse_proc = open_client_process(client)
- (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'")
+def collect_build_flags():
+ global clickhouse_client
+
result = []
- if clickhouse_proc.returncode == 0:
- if b'-fsanitize=thread' in stdout:
- result.append(BuildFlags.THREAD)
- elif b'-fsanitize=address' in stdout:
- result.append(BuildFlags.ADDRESS)
- elif b'-fsanitize=undefined' in stdout:
- result.append(BuildFlags.UNDEFINED)
- elif b'-fsanitize=memory' in stdout:
- result.append(BuildFlags.MEMORY)
- else:
- raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
+ value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'")
+ if '-fsanitize=thread' in value:
+ result.append(BuildFlags.THREAD)
+ elif '-fsanitize=address' in value:
+ result.append(BuildFlags.ADDRESS)
+ elif '-fsanitize=undefined' in value:
+ result.append(BuildFlags.UNDEFINED)
+ elif '-fsanitize=memory' in value:
+ result.append(BuildFlags.MEMORY)
- clickhouse_proc = open_client_process(client)
- (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'")
+ value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'")
+ if 'Debug' in value:
+ result.append(BuildFlags.DEBUG)
+ elif 'RelWithDebInfo' in value or 'Release' in value:
+ result.append(BuildFlags.RELEASE)
- if clickhouse_proc.returncode == 0:
- if b'Debug' in stdout:
- result.append(BuildFlags.DEBUG)
- elif b'RelWithDebInfo' in stdout or b'Release' in stdout:
- result.append(BuildFlags.RELEASE)
- else:
- raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
+ value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'")
+ if value in ('ON', '1'):
+ result.append(BuildFlags.UNBUNDLED)
- clickhouse_proc = open_client_process(client)
- (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'")
+ value = clickhouse_client.execute_one("SELECT value FROM system.settings WHERE name = 'default_database_engine'")
+ if value == 'Ordinary':
+ result.append(BuildFlags.ORDINARY_DATABASE)
- if clickhouse_proc.returncode == 0:
- if b'ON' in stdout or b'1' in stdout:
- result.append(BuildFlags.UNBUNDLED)
- else:
- raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
-
- clickhouse_proc = open_client_process(client)
- (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.settings WHERE name = 'default_database_engine'")
-
- if clickhouse_proc.returncode == 0:
- if b'Ordinary' in stdout:
- result.append(BuildFlags.ORDINARY_DATABASE)
- else:
- raise Exception("Cannot get information about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
-
- clickhouse_proc = open_client_process(client)
- (stdout, stderr) = clickhouse_proc.communicate(b"SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'")
-
- if clickhouse_proc.returncode == 0:
- if stdout == b'0\n':
- result.append(BuildFlags.POLYMORPHIC_PARTS)
- else:
- raise Exception("Cannot get inforamtion about build from server errorcode {}, stderr {}".format(clickhouse_proc.returncode, stderr))
+ value = int(clickhouse_client.execute_one("SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'"))
+ if value == 0:
+ result.append(BuildFlags.POLYMORPHIC_PARTS)
return result
@@ -1092,16 +1050,6 @@ def extract_key(key: str) -> str:
args.configserver + key)[1]
-def open_client_process(
- client_args: str,
- universal_newlines: bool = False,
- stderr_file: Optional[TextIO] = None):
- return Popen(
- shlex.split(client_args), stdin=PIPE, stdout=PIPE,
- stderr=stderr_file if stderr_file is not None else PIPE,
- universal_newlines=True if universal_newlines else None)
-
-
def do_run_tests(jobs, test_suite: TestSuite, parallel):
if jobs > 1 and len(test_suite.parallel_tests) > 0:
print("Found", len(test_suite.parallel_tests), "parallel tests and", len(test_suite.sequential_tests), "sequential tests")
@@ -1170,8 +1118,9 @@ def main(args):
global exit_code
global server_logs_level
global restarted_tests
+ global clickhouse_client
- if not check_server_started(args.client, args.server_check_retries):
+ if not check_server_started(args.server_check_retries):
msg = "Server is not responding. Cannot execute 'SELECT 1' query. \
If you are using split build, you have to specify -c option."
if args.hung_check:
@@ -1181,13 +1130,12 @@ def main(args):
print_stacktraces()
raise Exception(msg)
- args.build_flags = collect_build_flags(args.client)
+ args.build_flags = collect_build_flags()
if args.skip:
args.skip = set(args.skip)
base_dir = os.path.abspath(args.queries)
- tmp_dir = os.path.abspath(args.tmp)
# Keep same default values as in queries/shell_config.sh
os.environ.setdefault("CLICKHOUSE_BINARY", args.binary)
@@ -1218,17 +1166,12 @@ def main(args):
create_database_retries = 0
while create_database_retries < MAX_RETRIES:
start_time = datetime.now()
-
- client_cmd = args.client + " " + get_additional_client_options(args)
-
- clickhouse_proc_create = open_client_process(client_cmd, universal_newlines=True)
-
- (stdout, stderr) = clickhouse_proc_create.communicate(("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name)))
-
- total_time = (datetime.now() - start_time).total_seconds()
-
- if not need_retry(args, stdout, stderr, total_time):
- break
+ try:
+ clickhouse_client.execute("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name))
+ except Exception as e:
+ total_time = (datetime.now() - start_time).total_seconds()
+ if not need_retry_error(args, e, total_time):
+ break
create_database_retries += 1
if args.database and args.database != "test":
@@ -1255,18 +1198,14 @@ def main(args):
# Some queries may execute in background for some time after test was finished. This is normal.
for _ in range(1, 60):
- timeout, processlist = get_processlist(args)
- if timeout or not processlist:
+ processlist = get_processlist(args)
+ if processlist.empty:
break
sleep(1)
- if timeout or processlist:
- if processlist:
- print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"]))
- print(processlist)
- else:
- print(colored("Seems like server hung and cannot respond to queries", args, "red", attrs=["bold"]))
-
+ if not processlist.empty:
+ print(colored("\nFound hung queries in processlist:", args, "red", attrs=["bold"]))
+ print(processlist)
print_stacktraces()
exit_code.value = 1
@@ -1311,16 +1250,20 @@ def find_binary(name):
def get_additional_client_options(args):
if args.client_option:
return ' '.join('--' + option for option in args.client_option)
-
return ''
-
def get_additional_client_options_url(args):
if args.client_option:
return '&'.join(args.client_option)
-
return ''
+def get_additional_client_options_dict(args):
+ settings = {}
+ if args.client_option:
+ for key, value in map(lambda x: x.split('='), args.client_option):
+ settings[key] = value
+ return settings
+
if __name__ == '__main__':
stop_time = None
@@ -1439,14 +1382,24 @@ if __name__ == '__main__':
if args.configclient:
args.client += ' --config-file=' + args.configclient
- if os.getenv("CLICKHOUSE_HOST"):
- args.client += ' --host=' + os.getenv("CLICKHOUSE_HOST")
+ tcp_host = os.getenv("CLICKHOUSE_HOST")
+ if tcp_host is not None:
+ args.client += f' --host={tcp_host}'
+ else:
+ tcp_host = 'localhost'
- args.tcp_port = int(os.getenv("CLICKHOUSE_PORT_TCP", "9000"))
- args.client += f" --port={args.tcp_port}"
+ tcp_port = os.getenv("CLICKHOUSE_PORT_TCP")
+ if tcp_port is not None:
+ args.tcp_port = int(tcp_port)
+ args.client += f" --port={tcp_port}"
+ else:
+ args.tcp_port = 9000
- if os.getenv("CLICKHOUSE_DATABASE"):
- args.client += ' --database=' + os.getenv("CLICKHOUSE_DATABASE")
+ client_database = os.getenv("CLICKHOUSE_DATABASE")
+ if client_database is not None:
+ args.client += f' --database={client_database}'
+ else:
+ client_database = 'default'
if args.client_option:
# Set options for client
@@ -1474,4 +1427,13 @@ if __name__ == '__main__':
if args.jobs is None:
args.jobs = multiprocessing.cpu_count()
+ # configure pandas to make it more like Vertical format
+ pandas.options.display.max_columns = None
+ pandas.options.display.width = None
+
+ clickhouse_client = Client(host=tcp_host,
+ port=args.tcp_port,
+ database=client_database,
+ settings=get_additional_client_options_dict(args))
+
main(args)
From e2d6698244d43979b3fe2478dfdcd8dc3a91a0fd Mon Sep 17 00:00:00 2001
From: Azat Khuzhin
Date: Fri, 8 Oct 2021 00:07:05 +0300
Subject: [PATCH 066/264] clickhouse-test: do not use persistent connection for
simplicity (due to threads)
---
tests/clickhouse-test | 77 +++++++++++++++++++++----------------------
1 file changed, 38 insertions(+), 39 deletions(-)
diff --git a/tests/clickhouse-test b/tests/clickhouse-test
index e8c85a6ae79..6bbfa97ab66 100755
--- a/tests/clickhouse-test
+++ b/tests/clickhouse-test
@@ -86,6 +86,17 @@ class Client(clickhouse_driver.Client):
data = pandas.DataFrame(data=rows, columns=header)
return data
+# Helpers
+def make_clickhouse_client(base_args, *args, **kwargs):
+ return Client(host=base_args.tcp_host, port=base_args.tcp_port,
+ settings=get_additional_client_options_dict(base_args))
+def clickhouse_execute_one(base_args, *args, **kwargs):
+ return make_clickhouse_client(base_args).execute_one(*args, **kwargs)
+def clickhouse_execute(base_args, *args, **kwargs):
+ return make_clickhouse_client(base_args).execute(*args, **kwargs)
+def clickhouse_execute_pandas(base_args, *args, **kwargs):
+ return make_clickhouse_client(base_args).execute_pandas(*args, **kwargs)
+
class Terminated(KeyboardInterrupt):
pass
@@ -128,16 +139,14 @@ def get_db_engine(args, database_name):
def get_zookeeper_session_uptime(args):
- global clickhouse_client
-
try:
if args.replicated_database:
- return int(clickhouse_client.execute_one("""
+ return int(clickhouse_execute_one(args, """
SELECT min(materialize(zookeeperSessionUptime()))
FROM clusterAllReplicas('test_cluster_database_replicated', system.one)
"""))
else:
- return int(clickhouse_client.execute_one('SELECT zookeeperSessionUptime()'))
+ return int(clickhouse_execute_one(args, 'SELECT zookeeperSessionUptime()'))
except:
return None
@@ -167,15 +176,14 @@ def need_retry_error(args, error, total_time):
def get_processlist(args):
- global clickhouse_client
if args.replicated_database:
- return clickhouse_client.execute_pandas("""
+ return clickhouse_execute_pandas(args, """
SELECT materialize((hostName(), tcpPort())) as host, *
FROM clusterAllReplicas('test_cluster_database_replicated', system.processes)
WHERE query NOT LIKE '%system.processes%'
""")
else:
- return clickhouse_client.execute_pandas('SHOW PROCESSLIST')
+ return clickhouse_execute_pandas(args, 'SHOW PROCESSLIST')
# collect server stacktraces using gdb
@@ -342,7 +350,6 @@ class TestCase:
@staticmethod
def configure_testcase_args(args, case_file, suite_tmp_dir):
- global clickhouse_client
testcase_args = copy.deepcopy(args)
testcase_args.testcase_start_time = datetime.now()
@@ -363,7 +370,7 @@ class TestCase:
database = 'test_{suffix}'.format(suffix=random_str())
try:
- clickhouse_client.execute("CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename})
+ clickhouse_execute(args, "CREATE DATABASE " + database + get_db_engine(testcase_args, database), settings={'log_comment': testcase_basename})
except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError):
total_time = (datetime.now() - testcase_args.testcase_start_time).total_seconds()
return None, "", f"Timeout creating database {database} before test", total_time
@@ -536,12 +543,10 @@ class TestCase:
@staticmethod
def send_test_name_failed(suite: str, case: str) -> bool:
- global clickhouse_client
pid = os.getpid()
- clickhouse_client.execute(f"SELECT 'Running test {suite}/{case} from pid={pid}'")
+ clickhouse_execute(args, f"SELECT 'Running test {suite}/{case} from pid={pid}'")
def run_single_test(self, server_logs_level, client_options):
- global clickhouse_client
args = self.testcase_args
client = args.testcase_client
start_time = args.testcase_start_time
@@ -585,8 +590,10 @@ class TestCase:
if need_drop_database:
seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 20)
try:
- with clickhouse_client.connection.timeout_setter(seconds_left):
- clickhouse_client.execute("DROP DATABASE " + database)
+ client = make_clickhouse_client(args)
+ client.connection.force_connect()
+ with client.connection.timeout_setter(seconds_left):
+ client.execute("DROP DATABASE " + database)
except (TimeoutError, clickhouse_driver.errors.SocketTimeoutError):
total_time = (datetime.now() - start_time).total_seconds()
return None, "", f"Timeout dropping database {database} after test", total_time
@@ -793,8 +800,7 @@ class TestSuite:
@staticmethod
def readTestSuite(args, suite_dir_name: str):
def is_data_present():
- global clickhouse_client
- return int(clickhouse_client.execute_one('EXISTS TABLE test.hits'))
+ return int(clickhouse_execute_one(args, 'EXISTS TABLE test.hits'))
base_dir = os.path.abspath(args.queries)
tmp_dir = os.path.abspath(args.tmp)
@@ -827,7 +833,6 @@ class TestSuite:
stop_time = None
-clickhouse_client = None
exit_code = None
server_died = None
stop_tests_triggered_lock = None
@@ -957,14 +962,14 @@ def run_tests_array(all_tests_with_params):
server_logs_level = "warning"
-def check_server_started(retry_count):
- global clickhouse_client
+def check_server_started(args):
print("Connecting to ClickHouse server...", end='')
sys.stdout.flush()
+ retry_count = args.server_check_retries
while retry_count > 0:
try:
- clickhouse_client.execute('SELECT 1')
+ clickhouse_execute(args, 'SELECT 1')
print(" OK")
sys.stdout.flush()
return True
@@ -992,12 +997,10 @@ class BuildFlags():
POLYMORPHIC_PARTS = 'polymorphic-parts'
-def collect_build_flags():
- global clickhouse_client
-
+def collect_build_flags(args):
result = []
- value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'")
+ value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'")
if '-fsanitize=thread' in value:
result.append(BuildFlags.THREAD)
elif '-fsanitize=address' in value:
@@ -1007,21 +1010,21 @@ def collect_build_flags():
elif '-fsanitize=memory' in value:
result.append(BuildFlags.MEMORY)
- value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'")
+ value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'BUILD_TYPE'")
if 'Debug' in value:
result.append(BuildFlags.DEBUG)
elif 'RelWithDebInfo' in value or 'Release' in value:
result.append(BuildFlags.RELEASE)
- value = clickhouse_client.execute_one("SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'")
+ value = clickhouse_execute_one(args, "SELECT value FROM system.build_options WHERE name = 'UNBUNDLED'")
if value in ('ON', '1'):
result.append(BuildFlags.UNBUNDLED)
- value = clickhouse_client.execute_one("SELECT value FROM system.settings WHERE name = 'default_database_engine'")
+ value = clickhouse_execute_one(args, "SELECT value FROM system.settings WHERE name = 'default_database_engine'")
if value == 'Ordinary':
result.append(BuildFlags.ORDINARY_DATABASE)
- value = int(clickhouse_client.execute_one("SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'"))
+ value = int(clickhouse_execute_one(args, "SELECT value FROM system.merge_tree_settings WHERE name = 'min_bytes_for_wide_part'"))
if value == 0:
result.append(BuildFlags.POLYMORPHIC_PARTS)
@@ -1118,9 +1121,8 @@ def main(args):
global exit_code
global server_logs_level
global restarted_tests
- global clickhouse_client
- if not check_server_started(args.server_check_retries):
+ if not check_server_started(args):
msg = "Server is not responding. Cannot execute 'SELECT 1' query. \
If you are using split build, you have to specify -c option."
if args.hung_check:
@@ -1130,7 +1132,7 @@ def main(args):
print_stacktraces()
raise Exception(msg)
- args.build_flags = collect_build_flags()
+ args.build_flags = collect_build_flags(args)
if args.skip:
args.skip = set(args.skip)
@@ -1167,7 +1169,7 @@ def main(args):
while create_database_retries < MAX_RETRIES:
start_time = datetime.now()
try:
- clickhouse_client.execute("CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name))
+ clickhouse_execute(args, "CREATE DATABASE IF NOT EXISTS " + db_name + get_db_engine(args, db_name))
except Exception as e:
total_time = (datetime.now() - start_time).total_seconds()
if not need_retry_error(args, e, total_time):
@@ -1384,9 +1386,10 @@ if __name__ == '__main__':
tcp_host = os.getenv("CLICKHOUSE_HOST")
if tcp_host is not None:
+ args.tcp_host = tcp_host
args.client += f' --host={tcp_host}'
else:
- tcp_host = 'localhost'
+ args.tcp_host = 'localhost'
tcp_port = os.getenv("CLICKHOUSE_PORT_TCP")
if tcp_port is not None:
@@ -1398,8 +1401,9 @@ if __name__ == '__main__':
client_database = os.getenv("CLICKHOUSE_DATABASE")
if client_database is not None:
args.client += f' --database={client_database}'
+ args.client_database = client_database
else:
- client_database = 'default'
+ args.client_database = 'default'
if args.client_option:
# Set options for client
@@ -1431,9 +1435,4 @@ if __name__ == '__main__':
pandas.options.display.max_columns = None
pandas.options.display.width = None
- clickhouse_client = Client(host=tcp_host,
- port=args.tcp_port,
- database=client_database,
- settings=get_additional_client_options_dict(args))
-
main(args)
From f854065744f57607e23a0de3edcca1b06f06c11a Mon Sep 17 00:00:00 2001
From: Alexey
Date: Fri, 8 Oct 2021 05:05:12 +0000
Subject: [PATCH 067/264] buildID() description
---
docs/en/sql-reference/functions/other-functions.md | 6 ++++++
1 file changed, 6 insertions(+)
diff --git a/docs/en/sql-reference/functions/other-functions.md b/docs/en/sql-reference/functions/other-functions.md
index afcc9563b58..9828c91909b 100644
--- a/docs/en/sql-reference/functions/other-functions.md
+++ b/docs/en/sql-reference/functions/other-functions.md
@@ -699,6 +699,12 @@ If it is executed in the context of a distributed table, then it generates a nor
Returns the version of the server as a string.
If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
+## buildId() {#buildid}
+
+Returns the compiler build id of the running binary.
+If it is executed in the context of a distributed table, then it generates a normal column with values relevant to each shard. Otherwise it produces a constant value.
+
+
## blockNumber {#blocknumber}
Returns the sequence number of the data block where the row is located.
From d454a9affe73ee5844f19f8e85a4143c89c1d016 Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 8 Oct 2021 10:07:55 +0300
Subject: [PATCH 068/264] Update
docs/ru/getting-started/example-datasets/metrica.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/getting-started/example-datasets/metrica.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md
index 27105ca8488..4d862bae423 100644
--- a/docs/ru/getting-started/example-datasets/metrica.md
+++ b/docs/ru/getting-started/example-datasets/metrica.md
@@ -35,7 +35,7 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
``` bash
$ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv
-$ # теперь создадим таблицу
+# создадим таблицу hits_v1
$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
$ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
# for hits_100m_obfuscated
From 9d97a1263f4bd31c8520d9a54a35c8aa8877b982 Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 8 Oct 2021 10:08:02 +0300
Subject: [PATCH 069/264] Update
docs/ru/getting-started/example-datasets/metrica.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/getting-started/example-datasets/metrica.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md
index 4d862bae423..765642ce3ae 100644
--- a/docs/ru/getting-started/example-datasets/metrica.md
+++ b/docs/ru/getting-started/example-datasets/metrica.md
@@ -38,7 +38,7 @@ $ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=
# создадим таблицу hits_v1
$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
$ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
-# for hits_100m_obfuscated
+# создадим таблицу hits_100m_obfuscated
clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
$ # импортируем данные
From 2c5341df33a3410db2aed4f57d4429d681064186 Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 8 Oct 2021 10:08:25 +0300
Subject: [PATCH 070/264] Update docs/ru/sql-reference/functions/geo/h3.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/sql-reference/functions/geo/h3.md | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/docs/ru/sql-reference/functions/geo/h3.md b/docs/ru/sql-reference/functions/geo/h3.md
index e85236848f6..db96f0caa1d 100644
--- a/docs/ru/sql-reference/functions/geo/h3.md
+++ b/docs/ru/sql-reference/functions/geo/h3.md
@@ -6,7 +6,7 @@ toc_title: "Функции для работы с индексами H3"
[H3](https://eng.uber.com/h3/) — это система геокодирования, которая делит поверхность Земли на равные шестигранные ячейки. Система поддерживает иерархию (вложенность) ячеек, т.е. каждый "родительский" шестигранник может быть поделен на семь одинаковых вложенных "дочерних" шестигранников, и так далее.
-Уровень вложенности называется `разрешением` и может принимать значение от `0` до `15`, где `0` соответствует `базовым` ячейкам самого верхнего уровня (наиболее крупным).
+Уровень вложенности называется "разрешением" и может принимать значение от `0` до `15`, где `0` соответствует "базовым" ячейкам самого верхнего уровня (наиболее крупным).
Для каждой точки, имеющей широту и долготу, можно получить 64-битный индекс H3, соответствующий номеру шестигранной ячейки, где эта точка находится.
From 0fdbf867a45689308f0072a005a70b83757d8cc0 Mon Sep 17 00:00:00 2001
From: Roman Bug
Date: Fri, 8 Oct 2021 10:08:31 +0300
Subject: [PATCH 071/264] Update
docs/ru/getting-started/example-datasets/metrica.md
Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com>
---
docs/ru/getting-started/example-datasets/metrica.md | 4 ++--
1 file changed, 2 insertions(+), 2 deletions(-)
diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md
index 765642ce3ae..761e298fc54 100644
--- a/docs/ru/getting-started/example-datasets/metrica.md
+++ b/docs/ru/getting-started/example-datasets/metrica.md
@@ -41,9 +41,9 @@ $ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, Ja
# создадим таблицу hits_100m_obfuscated
clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
-$ # импортируем данные
+# импортируем данные
$ cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000
-$ # опционально можно оптимизировать таблицу
+# опционально можно оптимизировать таблицу
$ clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL"
$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
```
From d11cae26178fd114f69c763de4dbd18190175486 Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Fri, 8 Oct 2021 10:26:02 +0300
Subject: [PATCH 072/264] Remove `$ #` from metrica.md
---
docs/ru/getting-started/example-datasets/metrica.md | 10 +++++-----
1 file changed, 5 insertions(+), 5 deletions(-)
diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md
index 761e298fc54..0f5e7197fe5 100644
--- a/docs/ru/getting-started/example-datasets/metrica.md
+++ b/docs/ru/getting-started/example-datasets/metrica.md
@@ -14,7 +14,7 @@ toc_title: "Анонимизированные данные Яндекс.Мет
``` bash
$ curl -O https://datasets.clickhouse.com/hits/partitions/hits_v1.tar
$ tar xvf hits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse
-$ # убедитесь, что установлены корректные права доступа на файлы
+# убедитесь, что установлены корректные права доступа на файлы
$ sudo service clickhouse-server restart
$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
```
@@ -24,7 +24,7 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
``` bash
$ curl -O https://datasets.clickhouse.com/visits/partitions/visits_v1.tar
$ tar xvf visits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse
-$ # убедитесь, что установлены корректные права доступа на файлы
+# убедитесь, что установлены корректные права доступа на файлы
$ sudo service clickhouse-server restart
$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
```
@@ -52,12 +52,12 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
``` bash
$ curl https://datasets.clickhouse.com/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv
-$ # теперь создадим таблицу
+# теперь создадим таблицу
$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
$ clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
-$ # импортируем данные
+# импортируем данные
$ cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000
-$ # опционально можно оптимизировать таблицу
+# опционально можно оптимизировать таблицу
$ clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL"
$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
```
From bb55fb41c62b8412d2b245b58152d8a014c831b1 Mon Sep 17 00:00:00 2001
From: kssenii
Date: Fri, 8 Oct 2021 07:48:05 +0000
Subject: [PATCH 073/264] Add test
---
.../0_stateless/02047_client_exception.expect | 32 +++++++++++++++++++
.../02047_client_exception.reference | 0
2 files changed, 32 insertions(+)
create mode 100755 tests/queries/0_stateless/02047_client_exception.expect
create mode 100644 tests/queries/0_stateless/02047_client_exception.reference
diff --git a/tests/queries/0_stateless/02047_client_exception.expect b/tests/queries/0_stateless/02047_client_exception.expect
new file mode 100755
index 00000000000..120f8ef11e3
--- /dev/null
+++ b/tests/queries/0_stateless/02047_client_exception.expect
@@ -0,0 +1,32 @@
+#!/usr/bin/expect -f
+# Tags: no-unbundled, no-fasttest
+
+log_user 0
+set timeout 20
+match_max 100000
+
+# A default timeout action is to fail
+expect_after {
+ timeout {
+ exit 1
+ }
+}
+
+set basedir [file dirname $argv0]
+spawn bash -c "source $basedir/../shell_config.sh ; \$CLICKHOUSE_CLIENT_BINARY \$CLICKHOUSE_CLIENT_OPT --disable_suggestion"
+expect ":) "
+
+send -- "DROP TABLE IF EXISTS test_02047\r"
+expect "Ok."
+
+send -- "CREATE TABLE test_02047 (s Int32) ENGINE=Memory()\r"
+expect "Ok."
+
+send -- "INSERT INTO test_02047 SELECT 'f' \r"
+expect "Received exception from server"
+
+send -- "DROP TABLE test_02047\r"
+expect "Ok."
+
+send -- "\4"
+expect eof
diff --git a/tests/queries/0_stateless/02047_client_exception.reference b/tests/queries/0_stateless/02047_client_exception.reference
new file mode 100644
index 00000000000..e69de29bb2d
From b081f58db51a10a1d0fc6977cd4d9f108f12a265 Mon Sep 17 00:00:00 2001
From: MaxWk <610379995@qq.com>
Date: Fri, 8 Oct 2021 16:49:02 +0800
Subject: [PATCH 074/264] add alias
---
src/Storages/System/StorageSystemTables.cpp | 2 ++
1 file changed, 2 insertions(+)
diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp
index f1f7fa4fa08..254e6f77e0c 100644
--- a/src/Storages/System/StorageSystemTables.cpp
+++ b/src/Storages/System/StorageSystemTables.cpp
@@ -59,6 +59,8 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_)
{"lifetime_bytes", std::make_shared(std::make_shared())},
{"comment", std::make_shared()},
{"has_own_data", std::make_shared()},
+ }, {
+ {"table", std::make_shared(), "name"}
}));
setInMemoryMetadata(storage_metadata);
}
From 383f3a3a20ff7268b4c2fad14d0e20f266682b77 Mon Sep 17 00:00:00 2001
From: vesslanjin
Date: Fri, 8 Oct 2021 09:19:58 -0400
Subject: [PATCH 075/264] =?UTF-8?q?Remove=C2=A0branchy=C2=A0code=C2=A0in?=
=?UTF-8?q?=C2=A0filter=C2=A0operation=C2=A0with=C2=A0a=C2=A0better=C2=A0i?=
=?UTF-8?q?mplementation=C2=A0with=C2=A0popcnt/ctz=C2=A0which=C2=A0have?=
=?UTF-8?q?=C2=A0better=C2=A0performance?=
MIME-Version: 1.0
Content-Type: text/plain; charset=UTF-8
Content-Transfer-Encoding: 8bit
Co-authored-by: Zhu Jasper
---
src/Columns/ColumnFixedString.cpp | 22 +++++++++-------------
src/Columns/ColumnVector.cpp | 11 +++++++----
src/Columns/ColumnsCommon.cpp | 11 +++++++----
3 files changed, 23 insertions(+), 21 deletions(-)
diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp
index ce39ab0994c..4bfc6513263 100644
--- a/src/Columns/ColumnFixedString.cpp
+++ b/src/Columns/ColumnFixedString.cpp
@@ -231,7 +231,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
const UInt8 * filt_end = filt_pos + col_size;
const UInt8 * data_pos = chars.data();
-#ifdef __SSE2__
+#if defined(__SSE2__) && defined(__POPCNT__)
/** A slightly more optimized version.
* Based on the assumption that often pieces of consecutive values
* completely pass or do not pass the filter.
@@ -251,28 +251,24 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
if (0 == mask)
{
/// Nothing is inserted.
- data_pos += chars_per_simd_elements;
}
else if (0xFFFF == mask)
{
res->chars.insert(data_pos, data_pos + chars_per_simd_elements);
- data_pos += chars_per_simd_elements;
}
else
{
size_t res_chars_size = res->chars.size();
- for (size_t i = 0; i < SIMD_BYTES; ++i)
- {
- if (filt_pos[i])
- {
- res->chars.resize(res_chars_size + n);
- memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos, n);
- res_chars_size += n;
- }
- data_pos += n;
+ size_t pcnt = __builtin_popcount(mask);
+ for(size_t j = 0; j < pcnt; j++) {
+ size_t index = __builtin_ctz(mask);
+ res->chars.resize(res_chars_size + n);
+ memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos+index*n, n);
+ res_chars_size += n;
+ mask = mask & (mask-1);
}
}
-
+ data_pos += chars_per_simd_elements;
filt_pos += SIMD_BYTES;
}
#endif
diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp
index 7f3cdaeec7f..ff84204a7cb 100644
--- a/src/Columns/ColumnVector.cpp
+++ b/src/Columns/ColumnVector.cpp
@@ -311,7 +311,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s
const UInt8 * filt_end = filt_pos + size;
const T * data_pos = data.data();
-#ifdef __SSE2__
+#if defined(__SSE2__) && defined(__POPCNT__)
/** A slightly more optimized version.
* Based on the assumption that often pieces of consecutive values
* completely pass or do not pass the filter.
@@ -337,9 +337,12 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s
}
else
{
- for (size_t i = 0; i < SIMD_BYTES; ++i)
- if (filt_pos[i])
- res_data.push_back(data_pos[i]);
+ size_t pcnt = __builtin_popcount(mask);
+ for(size_t j = 0; j < pcnt; j++) {
+ size_t index = __builtin_ctz(mask);
+ res_data.push_back(data_pos[index]);
+ mask = mask & (mask-1);
+ }
}
filt_pos += SIMD_BYTES;
diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp
index 41933ed08ed..5c0214054b2 100644
--- a/src/Columns/ColumnsCommon.cpp
+++ b/src/Columns/ColumnsCommon.cpp
@@ -229,7 +229,7 @@ namespace
memcpy(&res_elems[elems_size_old], &src_elems[arr_offset], arr_size * sizeof(T));
};
- #ifdef __SSE2__
+ #if defined(__SSE2__) && defined(__POPCNT__)
const __m128i zero_vec = _mm_setzero_si128();
static constexpr size_t SIMD_BYTES = 16;
const auto * filt_end_aligned = filt_pos + size / SIMD_BYTES * SIMD_BYTES;
@@ -262,9 +262,12 @@ namespace
}
else
{
- for (size_t i = 0; i < SIMD_BYTES; ++i)
- if (filt_pos[i])
- copy_array(offsets_pos + i);
+ size_t pcnt = __builtin_popcount(mask);
+ for(size_t j = 0; j < pcnt; j++) {
+ size_t index = __builtin_ctz(mask);
+ copy_array(offsets_pos + index);
+ mask = mask & (mask-1);
+ }
}
filt_pos += SIMD_BYTES;
From 35feb082365eb0517f2c58659507de34ed47d3af Mon Sep 17 00:00:00 2001
From: Vladimir C
Date: Fri, 8 Oct 2021 12:03:55 +0300
Subject: [PATCH 076/264] Whitespace changes
---
src/Columns/ColumnFixedString.cpp | 7 ++++---
src/Columns/ColumnVector.cpp | 5 +++--
src/Columns/ColumnsCommon.cpp | 5 +++--
3 files changed, 10 insertions(+), 7 deletions(-)
diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp
index 4bfc6513263..1080f29b5aa 100644
--- a/src/Columns/ColumnFixedString.cpp
+++ b/src/Columns/ColumnFixedString.cpp
@@ -260,12 +260,13 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result
{
size_t res_chars_size = res->chars.size();
size_t pcnt = __builtin_popcount(mask);
- for(size_t j = 0; j < pcnt; j++) {
+ for (size_t j = 0; j < pcnt; ++j)
+ {
size_t index = __builtin_ctz(mask);
res->chars.resize(res_chars_size + n);
- memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos+index*n, n);
+ memcpySmallAllowReadWriteOverflow15(&res->chars[res_chars_size], data_pos + index * n, n);
res_chars_size += n;
- mask = mask & (mask-1);
+ mask = mask & (mask - 1);
}
}
data_pos += chars_per_simd_elements;
diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp
index ff84204a7cb..e7c56a57d51 100644
--- a/src/Columns/ColumnVector.cpp
+++ b/src/Columns/ColumnVector.cpp
@@ -338,10 +338,11 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s
else
{
size_t pcnt = __builtin_popcount(mask);
- for(size_t j = 0; j < pcnt; j++) {
+ for (size_t j = 0; j < pcnt; ++j)
+ {
size_t index = __builtin_ctz(mask);
res_data.push_back(data_pos[index]);
- mask = mask & (mask-1);
+ mask = mask & (mask - 1);
}
}
diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp
index 5c0214054b2..b4614abe490 100644
--- a/src/Columns/ColumnsCommon.cpp
+++ b/src/Columns/ColumnsCommon.cpp
@@ -263,10 +263,11 @@ namespace
else
{
size_t pcnt = __builtin_popcount(mask);
- for(size_t j = 0; j < pcnt; j++) {
+ for (size_t j = 0; j < pcnt; ++j)
+ {
size_t index = __builtin_ctz(mask);
copy_array(offsets_pos + index);
- mask = mask & (mask-1);
+ mask = mask & (mask - 1);
}
}
From a4d1ad61d0e220b6c215eaffd4cf3531e4c01423 Mon Sep 17 00:00:00 2001
From: romanzhukov
Date: Fri, 8 Oct 2021 12:09:12 +0300
Subject: [PATCH 077/264] Remove $ symbols in bash commands.
---
.../example-datasets/metrica.md | 40 +++++++++----------
1 file changed, 20 insertions(+), 20 deletions(-)
diff --git a/docs/ru/getting-started/example-datasets/metrica.md b/docs/ru/getting-started/example-datasets/metrica.md
index 0f5e7197fe5..15c0cc14517 100644
--- a/docs/ru/getting-started/example-datasets/metrica.md
+++ b/docs/ru/getting-started/example-datasets/metrica.md
@@ -12,21 +12,21 @@ toc_title: "Анонимизированные данные Яндекс.Мет
**Скачивание и импортирование партиций hits:**
``` bash
-$ curl -O https://datasets.clickhouse.com/hits/partitions/hits_v1.tar
-$ tar xvf hits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse
+curl -O https://datasets.clickhouse.com/hits/partitions/hits_v1.tar
+tar xvf hits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse
# убедитесь, что установлены корректные права доступа на файлы
-$ sudo service clickhouse-server restart
-$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
+sudo service clickhouse-server restart
+clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
```
**Скачивание и импортирование партиций visits:**
``` bash
-$ curl -O https://datasets.clickhouse.com/visits/partitions/visits_v1.tar
-$ tar xvf visits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse
+curl -O https://datasets.clickhouse.com/visits/partitions/visits_v1.tar
+tar xvf visits_v1.tar -C /var/lib/clickhouse # путь к папке с данными ClickHouse
# убедитесь, что установлены корректные права доступа на файлы
-$ sudo service clickhouse-server restart
-$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
+sudo service clickhouse-server restart
+clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
```
## Получение таблиц из сжатых tsv-файлов {#poluchenie-tablits-iz-szhatykh-tsv-failov}
@@ -34,32 +34,32 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
**Скачивание и импортирование hits из сжатого tsv-файла**
``` bash
-$ curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv
+curl https://datasets.clickhouse.com/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv
# создадим таблицу hits_v1
-$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
-$ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
+clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
+clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
# создадим таблицу hits_100m_obfuscated
clickhouse-client --query="CREATE TABLE hits_100m_obfuscated (WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, Refresh UInt8, RefererCategoryID UInt16, RefererRegionID UInt32, URLCategoryID UInt16, URLRegionID UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, OriginalURL String, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), LocalEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, RemoteIP UInt32, WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming UInt32, DNSTiming UInt32, ConnectTiming UInt32, ResponseStartTiming UInt32, ResponseEndTiming UInt32, FetchTiming UInt32, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
# импортируем данные
-$ cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000
+cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000
# опционально можно оптимизировать таблицу
-$ clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL"
-$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
+clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL"
+clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1"
```
**Скачивание и импортирование visits из сжатого tsv-файла**
``` bash
-$ curl https://datasets.clickhouse.com/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv
+curl https://datasets.clickhouse.com/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv
# теперь создадим таблицу
-$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
-$ clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
+clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets"
+clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(Sign) PARTITION BY toYYYYMM(StartDate) ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192"
# импортируем данные
-$ cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000
+cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000
# опционально можно оптимизировать таблицу
-$ clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL"
-$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
+clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL"
+clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
```
## Запросы {#zaprosy}
From 525df998934368aedb3baac72645453b080cf01d Mon Sep 17 00:00:00 2001
From: Vladimir C
Date: Fri, 8 Oct 2021 12:10:19 +0300
Subject: [PATCH 078/264] Whitespace fix
---
src/Columns/ColumnVector.cpp | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp
index e7c56a57d51..61ba2074bd7 100644
--- a/src/Columns/ColumnVector.cpp
+++ b/src/Columns/ColumnVector.cpp
@@ -343,7 +343,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s
size_t index = __builtin_ctz(mask);
res_data.push_back(data_pos[index]);
mask = mask & (mask - 1);
- }
+ }
}
filt_pos += SIMD_BYTES;
From 3acdcc6ec5b31d3b5e7c33e0e49df99b39dbdda4 Mon Sep 17 00:00:00 2001
From: Nikolai Kochetov
Date: Fri, 8 Oct 2021 15:03:31 +0300
Subject: [PATCH 079/264] Update example
---
src/Core/examples/coro.cpp | 44 ++++++++++++++------------------------
1 file changed, 16 insertions(+), 28 deletions(-)
diff --git a/src/Core/examples/coro.cpp b/src/Core/examples/coro.cpp
index c8e2f7418e4..a9728314495 100644
--- a/src/Core/examples/coro.cpp
+++ b/src/Core/examples/coro.cpp
@@ -12,37 +12,25 @@
#include
#if defined(__clang__)
-
#include
-template
-using coroutine_handle = std::experimental::coroutine_handle;
-
-using default_coroutine_handle = std::experimental::coroutine_handle<>;
-
-using suspend_never = std::experimental::suspend_never;
-using suspend_always = std::experimental::suspend_always;
+namespace std
+{
+ using namespace experimental::coroutines_v1;
+}
#else
-
#include
-
-template
-using coroutine_handle = std::coroutine_handle;
-
-using default_coroutine_handle = std::coroutine_handle<>;
-
-using suspend_never = std::suspend_never;
-using suspend_always = std::suspend_always;
-
+#pragma GCC diagnostic push
+#pragma GCC diagnostic ignored "-Wzero-as-null-pointer-constant"
#endif
template
-struct suspend_never_val
+struct suspend_value
{
constexpr bool await_ready() const noexcept { return true; }
- constexpr void await_suspend(default_coroutine_handle) const noexcept {}
+ constexpr void await_suspend(std::coroutine_handle<>) const noexcept {}
constexpr T await_resume() const noexcept
{
std::cout << " ret " << val << std::endl;
@@ -57,10 +45,10 @@ struct resumable
{
struct promise_type
{
- using coro_handle = coroutine_handle;
+ using coro_handle = std::coroutine_handle;
auto get_return_object() { return coro_handle::from_promise(*this); }
- auto initial_suspend() { return suspend_never(); }
- auto final_suspend() noexcept { return suspend_never_val{*r->value}; }
+ auto initial_suspend() { return std::suspend_never(); }
+ auto final_suspend() noexcept { return suspend_value{*r->value}; }
//void return_void() {}
void return_value(T value_) { r->value = value_; }
void unhandled_exception()
@@ -76,7 +64,7 @@ struct resumable
resumable * r = nullptr;
};
- using coro_handle = coroutine_handle;
+ using coro_handle = std::coroutine_handle;
bool await_ready() const noexcept { return false; }
void await_suspend(coro_handle g) noexcept
@@ -148,16 +136,16 @@ private:
std::exception_ptr exception;
};
-resumable boo(std::string tag)
+resumable boo([[maybe_unused]] std::string tag)
{
std::cout << "x" << std::endl;
- co_await suspend_always();
+ co_await std::suspend_always();
std::cout << StackTrace().toString();
std::cout << "y" << std::endl;
co_return 1;
}
-resumable bar(std::string tag)
+resumable bar([[maybe_unused]] std::string tag)
{
std::cout << "a" << std::endl;
int res1 = co_await boo("boo1");
@@ -169,7 +157,7 @@ resumable bar(std::string tag)
co_return res1 + res2; // 1 + 1 = 2
}
-resumable foo(std::string tag) {
+resumable foo([[maybe_unused]] std::string tag) {
std::cout << "Hello" << std::endl;
auto res1 = co_await bar("bar1");
std::cout << "Coro " << res1 << std::endl;
From b8e77a0c62ca664e65f545a39c1132edb14995e1 Mon Sep 17 00:00:00 2001
From: Nikolai Kochetov
Date: Fri, 8 Oct 2021 15:06:09 +0300
Subject: [PATCH 080/264] Add -fcoroutines for g++
---
CMakeLists.txt | 4 ++++
1 file changed, 4 insertions(+)
diff --git a/CMakeLists.txt b/CMakeLists.txt
index 66df4d3124a..685b2c25a0d 100644
--- a/CMakeLists.txt
+++ b/CMakeLists.txt
@@ -336,6 +336,10 @@ if (COMPILER_GCC OR COMPILER_CLANG)
set(COMPILER_FLAGS "${COMPILER_FLAGS} -falign-functions=32")
endif ()
+if (COMPILER_GCC)
+ set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -fcoroutines")
+endif ()
+
# Compiler-specific coverage flags e.g. -fcoverage-mapping for gcc
option(WITH_COVERAGE "Profile the resulting binary/binaries" OFF)
From f0e193d5a51725f524945adf55e85b80413f9c77 Mon Sep 17 00:00:00 2001
From: MaxWk <610379995@qq.com>
Date: Fri, 8 Oct 2021 20:25:37 +0800
Subject: [PATCH 081/264] add database alias
---
src/Storages/System/StorageSystemDatabases.cpp | 7 +++++++
src/Storages/System/StorageSystemDatabases.h | 2 ++
2 files changed, 9 insertions(+)
diff --git a/src/Storages/System/StorageSystemDatabases.cpp b/src/Storages/System/StorageSystemDatabases.cpp
index a781f44e9b2..fb7a1bc59b8 100644
--- a/src/Storages/System/StorageSystemDatabases.cpp
+++ b/src/Storages/System/StorageSystemDatabases.cpp
@@ -21,6 +21,13 @@ NamesAndTypesList StorageSystemDatabases::getNamesAndTypes()
};
}
+NamesAndAliases StorageSystemDatabases::getNamesAndAliases()
+{
+ return {
+ {"database", std::make_shared(), "name"}
+ };
+}
+
void StorageSystemDatabases::fillData(MutableColumns & res_columns, ContextPtr context, const SelectQueryInfo &) const
{
const auto access = context->getAccess();
diff --git a/src/Storages/System/StorageSystemDatabases.h b/src/Storages/System/StorageSystemDatabases.h
index 4aaae1bfd7f..3de0da126d4 100644
--- a/src/Storages/System/StorageSystemDatabases.h
+++ b/src/Storages/System/StorageSystemDatabases.h
@@ -23,6 +23,8 @@ public:
static NamesAndTypesList getNamesAndTypes();
+ static NamesAndAliases getNamesAndAliases();
+
protected:
using IStorageSystemOneBlock::IStorageSystemOneBlock;
From f47e7aa8b84cb9b851e96a034a2bf6a9d5298dd3 Mon Sep 17 00:00:00 2001
From: MaxWk <610379995@qq.com>
Date: Fri, 8 Oct 2021 20:27:13 +0800
Subject: [PATCH 082/264] add test
---
.../02047_alias_for_table_and_database_name.reference | 2 ++
.../0_stateless/02047_alias_for_table_and_database_name.sql | 2 ++
2 files changed, 4 insertions(+)
create mode 100644 tests/queries/0_stateless/02047_alias_for_table_and_database_name.reference
create mode 100644 tests/queries/0_stateless/02047_alias_for_table_and_database_name.sql
diff --git a/tests/queries/0_stateless/02047_alias_for_table_and_database_name.reference b/tests/queries/0_stateless/02047_alias_for_table_and_database_name.reference
new file mode 100644
index 00000000000..324e7ff9ab8
--- /dev/null
+++ b/tests/queries/0_stateless/02047_alias_for_table_and_database_name.reference
@@ -0,0 +1,2 @@
+numbers numbers
+default default
diff --git a/tests/queries/0_stateless/02047_alias_for_table_and_database_name.sql b/tests/queries/0_stateless/02047_alias_for_table_and_database_name.sql
new file mode 100644
index 00000000000..2fabd2affd4
--- /dev/null
+++ b/tests/queries/0_stateless/02047_alias_for_table_and_database_name.sql
@@ -0,0 +1,2 @@
+SELECT name,table from system.tables where database = 'system' and name = 'numbers';
+SELECt name,database from system.databases where name = 'default';
From 7fa3257bb6b5bf5ef8576f70c4657335442a0c70 Mon Sep 17 00:00:00 2001
From: Artur Filatenkov <613623@mail.ru>
Date: Fri, 8 Oct 2021 15:37:24 +0300
Subject: [PATCH 083/264] correct test
---
tests/integration/test_settings_profile/test.py | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/tests/integration/test_settings_profile/test.py b/tests/integration/test_settings_profile/test.py
index 7be0b395764..78efe7ffcad 100644
--- a/tests/integration/test_settings_profile/test.py
+++ b/tests/integration/test_settings_profile/test.py
@@ -201,7 +201,7 @@ def test_show_profiles():
assert instance.query("SHOW CREATE PROFILE xyz") == "CREATE SETTINGS PROFILE xyz\n"
assert instance.query(
- "SHOW CREATE SETTINGS PROFILE default") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n"
+ "SHOW CREATE SETTINGS PROFILE default") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\', add_http_cors_header = 1\n"
assert instance.query(
"SHOW CREATE PROFILES") == "CREATE SETTINGS PROFILE default SETTINGS max_memory_usage = 10000000000, load_balancing = \\'random\\'\n" \
"CREATE SETTINGS PROFILE readonly SETTINGS readonly = 1\n" \
From 340b53ef853348758c9042b16a8599120ebc8d22 Mon Sep 17 00:00:00 2001
From: Nikolai Kochetov
Date: Fri, 8 Oct 2021 17:03:54 +0300
Subject: [PATCH 084/264] Remove some more streams.
---
programs/benchmark/Benchmark.cpp | 17 ++--
programs/copier/ClusterCopier.cpp | 1 -
programs/copier/Internals.h | 2 +-
src/Client/ClientBase.cpp | 11 ++-
src/Client/ClientBase.h | 7 +-
src/Columns/ColumnAggregateFunction.cpp | 2 +-
src/Columns/ColumnArray.cpp | 2 +-
src/Columns/ColumnDecimal.cpp | 2 +-
src/Columns/ColumnFixedString.cpp | 2 +-
src/Columns/ColumnLowCardinality.cpp | 2 +-
src/Columns/ColumnMap.cpp | 2 +-
src/Columns/ColumnNullable.cpp | 2 +-
src/Columns/ColumnString.cpp | 2 +-
src/Columns/ColumnTuple.cpp | 2 +-
src/Columns/ColumnVector.cpp | 2 +-
.../ExpressionBlockInputStream.cpp | 39 ----------
src/DataStreams/ExpressionBlockInputStream.h | 52 -------------
...wOutputStream.cpp => InternalTextLogs.cpp} | 9 +--
...gsRowOutputStream.h => InternalTextLogs.h} | 9 +--
.../MaterializingBlockInputStream.cpp | 28 -------
.../MaterializingBlockInputStream.h | 21 -----
src/DataStreams/RemoteBlockInputStream.cpp | 69 ----------------
src/DataStreams/RemoteBlockInputStream.h | 78 -------------------
src/Dictionaries/MongoDBDictionarySource.cpp | 2 +-
.../PostgreSQLDictionarySource.cpp | 2 +-
src/Interpreters/InterpreterInsertQuery.cpp | 2 +-
src/Interpreters/InterpreterSelectQuery.cpp | 5 +-
src/Interpreters/MutationsInterpreter.cpp | 2 +-
src/Interpreters/ThreadStatusExt.cpp | 2 +-
.../getHeaderForProcessingStage.cpp | 8 +-
.../Algorithms/CollapsingSortedAlgorithm.h | 2 +-
.../Algorithms/MergingSortedAlgorithm.cpp | 2 +-
.../Algorithms/ReplacingSortedAlgorithm.h | 2 +-
.../Algorithms/VersionedCollapsingAlgorithm.h | 2 +-
.../Merges/MergingSortedTransform.cpp | 2 +-
.../Sources/SourceFromInputStream.cpp | 5 --
.../Transforms/ColumnGathererTransform.cpp} | 2 +-
.../Transforms/ColumnGathererTransform.h} | 0
.../Transforms/DistinctSortedTransform.cpp} | 2 +-
.../Transforms/DistinctSortedTransform.h} | 0
.../Transforms}/MongoDBSource.cpp | 0
.../Transforms}/MongoDBSource.h | 0
.../Transforms}/PostgreSQLSource.cpp | 0
.../Transforms}/PostgreSQLSource.h | 0
.../Transforms/buildPushingToViewsChain.cpp} | 2 +-
.../Transforms/buildPushingToViewsChain.h} | 0
src/Storages/MergeTree/MergeTask.cpp | 5 +-
src/Storages/MergeTree/MergeTask.h | 2 +-
.../MergeTree/MergeTreeDataMergerMutator.cpp | 6 +-
src/Storages/MergeTree/MutateTask.cpp | 6 +-
.../PostgreSQLReplicationHandler.cpp | 2 +-
src/Storages/StorageMongoDB.cpp | 2 +-
src/Storages/StoragePostgreSQL.cpp | 2 +-
src/Storages/StorageReplicatedMergeTree.cpp | 1 -
src/Storages/StorageS3Cluster.cpp | 2 +-
src/Storages/getStructureOfRemoteTable.cpp | 12 +--
.../tests/gtest_row_source_bits_test.cpp | 2 +-
src/TableFunctions/TableFunctionS3Cluster.cpp | 2 +-
58 files changed, 73 insertions(+), 378 deletions(-)
delete mode 100644 src/DataStreams/ExpressionBlockInputStream.cpp
delete mode 100644 src/DataStreams/ExpressionBlockInputStream.h
rename src/DataStreams/{InternalTextLogsRowOutputStream.cpp => InternalTextLogs.cpp} (94%)
rename src/DataStreams/{InternalTextLogsRowOutputStream.h => InternalTextLogs.h} (57%)
delete mode 100644 src/DataStreams/MaterializingBlockInputStream.cpp
delete mode 100644 src/DataStreams/MaterializingBlockInputStream.h
delete mode 100644 src/DataStreams/RemoteBlockInputStream.cpp
delete mode 100644 src/DataStreams/RemoteBlockInputStream.h
rename src/{DataStreams/ColumnGathererStream.cpp => Processors/Transforms/ColumnGathererTransform.cpp} (98%)
rename src/{DataStreams/ColumnGathererStream.h => Processors/Transforms/ColumnGathererTransform.h} (100%)
rename src/{DataStreams/DistinctSortedBlockInputStream.cpp => Processors/Transforms/DistinctSortedTransform.cpp} (98%)
rename src/{DataStreams/DistinctSortedBlockInputStream.h => Processors/Transforms/DistinctSortedTransform.h} (100%)
rename src/{DataStreams => Processors/Transforms}/MongoDBSource.cpp (100%)
rename src/{DataStreams => Processors/Transforms}/MongoDBSource.h (100%)
rename src/{DataStreams => Processors/Transforms}/PostgreSQLSource.cpp (100%)
rename src/{DataStreams => Processors/Transforms}/PostgreSQLSource.h (100%)
rename src/{DataStreams/PushingToViewsBlockOutputStream.cpp => Processors/Transforms/buildPushingToViewsChain.cpp} (99%)
rename src/{DataStreams/PushingToViewsBlockOutputStream.h => Processors/Transforms/buildPushingToViewsChain.h} (100%)
diff --git a/programs/benchmark/Benchmark.cpp b/programs/benchmark/Benchmark.cpp
index be57a3b92a0..caa0a87bde2 100644
--- a/programs/benchmark/Benchmark.cpp
+++ b/programs/benchmark/Benchmark.cpp
@@ -28,7 +28,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
@@ -424,20 +424,19 @@ private:
if (reconnect)
connection.disconnect();
- RemoteBlockInputStream stream(
+ RemoteQueryExecutor executor(
connection, query, {}, global_context, nullptr, Scalars(), Tables(), query_processing_stage);
if (!query_id.empty())
- stream.setQueryId(query_id);
+ executor.setQueryId(query_id);
Progress progress;
- stream.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });
+ executor.setProgressCallback([&progress](const Progress & value) { progress.incrementPiecewiseAtomically(value); });
- stream.readPrefix();
- while (Block block = stream.read());
+ BlockStreamProfileInfo info;
+ while (Block block = executor.read())
+ info.update(block);
- stream.readSuffix();
-
- const BlockStreamProfileInfo & info = stream.getProfileInfo();
+ executor.finish();
double seconds = watch.elapsedSeconds();
diff --git a/programs/copier/ClusterCopier.cpp b/programs/copier/ClusterCopier.cpp
index 30b99b69351..60e1590d3bd 100644
--- a/programs/copier/ClusterCopier.cpp
+++ b/programs/copier/ClusterCopier.cpp
@@ -14,7 +14,6 @@
#include
#include
#include
-#include
namespace DB
{
diff --git a/programs/copier/Internals.h b/programs/copier/Internals.h
index 54e7e7719e0..45531e750b9 100644
--- a/programs/copier/Internals.h
+++ b/programs/copier/Internals.h
@@ -49,7 +49,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp
index cde5a5f9977..99cadabfaea 100644
--- a/src/Client/ClientBase.cpp
+++ b/src/Client/ClientBase.cpp
@@ -48,7 +48,7 @@
#include
#include
-#include
+#include
namespace fs = std::filesystem;
@@ -95,6 +95,9 @@ void interruptSignalHandler(int signum)
_exit(signum);
}
+ClientBase::~ClientBase() = default;
+ClientBase::ClientBase() = default;
+
void ClientBase::setupSignalHandler()
{
exit_on_signal.test_and_set();
@@ -393,8 +396,7 @@ void ClientBase::initLogsOutputStream()
}
}
- logs_out_stream = std::make_shared(*wb, stdout_is_a_tty);
- logs_out_stream->writePrefix();
+ logs_out_stream = std::make_unique(*wb, stdout_is_a_tty);
}
}
@@ -641,9 +643,6 @@ void ClientBase::onEndOfStream()
if (block_out_stream)
block_out_stream->writeSuffix();
- if (logs_out_stream)
- logs_out_stream->writeSuffix();
-
resetOutput();
if (is_interactive && !written_first_block)
diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h
index bf9e8fdfe47..b122803e1db 100644
--- a/src/Client/ClientBase.h
+++ b/src/Client/ClientBase.h
@@ -32,12 +32,17 @@ enum MultiQueryProcessingStage
void interruptSignalHandler(int signum);
+class InternalTextLogs;
+
class ClientBase : public Poco::Util::Application
{
public:
using Arguments = std::vector;
+ ClientBase();
+ ~ClientBase() override;
+
void init(int argc, char ** argv);
protected:
@@ -177,7 +182,7 @@ protected:
/// The user could specify special file for server logs (stderr by default)
std::unique_ptr out_logs_buf;
String server_logs_file;
- BlockOutputStreamPtr logs_out_stream;
+ std::unique_ptr logs_out_stream;
String home_path;
String history_file; /// Path to a file containing command history.
diff --git a/src/Columns/ColumnAggregateFunction.cpp b/src/Columns/ColumnAggregateFunction.cpp
index 5224ff0fbdb..cd1185d99db 100644
--- a/src/Columns/ColumnAggregateFunction.cpp
+++ b/src/Columns/ColumnAggregateFunction.cpp
@@ -2,7 +2,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/Columns/ColumnArray.cpp b/src/Columns/ColumnArray.cpp
index 8e613ae93c2..5e40b89cc7e 100644
--- a/src/Columns/ColumnArray.cpp
+++ b/src/Columns/ColumnArray.cpp
@@ -13,7 +13,7 @@
#include
#include
-#include
+#include
#include
#include
diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp
index d50138bc582..d8c5ced4b6b 100644
--- a/src/Columns/ColumnDecimal.cpp
+++ b/src/Columns/ColumnDecimal.cpp
@@ -16,7 +16,7 @@
#include
#include
#include
-#include
+#include
template bool decimalLess(T x, T y, UInt32 x_scale, UInt32 y_scale);
diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp
index ce39ab0994c..2bfd46a9607 100644
--- a/src/Columns/ColumnFixedString.cpp
+++ b/src/Columns/ColumnFixedString.cpp
@@ -2,7 +2,7 @@
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/Columns/ColumnLowCardinality.cpp b/src/Columns/ColumnLowCardinality.cpp
index 6f7bc58406d..5b33baab345 100644
--- a/src/Columns/ColumnLowCardinality.cpp
+++ b/src/Columns/ColumnLowCardinality.cpp
@@ -2,7 +2,7 @@
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/Columns/ColumnMap.cpp b/src/Columns/ColumnMap.cpp
index 03addee4f8d..4374ca30c99 100644
--- a/src/Columns/ColumnMap.cpp
+++ b/src/Columns/ColumnMap.cpp
@@ -1,7 +1,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/Columns/ColumnNullable.cpp b/src/Columns/ColumnNullable.cpp
index d4d62f9580c..4b3340a6b50 100644
--- a/src/Columns/ColumnNullable.cpp
+++ b/src/Columns/ColumnNullable.cpp
@@ -8,7 +8,7 @@
#include
#include
#include
-#include
+#include
namespace DB
diff --git a/src/Columns/ColumnString.cpp b/src/Columns/ColumnString.cpp
index bb34f524fdf..2beb9add318 100644
--- a/src/Columns/ColumnString.cpp
+++ b/src/Columns/ColumnString.cpp
@@ -4,7 +4,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp
index f053c32fda1..d157f18bf32 100644
--- a/src/Columns/ColumnTuple.cpp
+++ b/src/Columns/ColumnTuple.cpp
@@ -3,7 +3,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp
index 7f3cdaeec7f..56c2cef3243 100644
--- a/src/Columns/ColumnVector.cpp
+++ b/src/Columns/ColumnVector.cpp
@@ -4,7 +4,7 @@
#include
#include
#include
-#include
+#include
#include
#include
#include
diff --git a/src/DataStreams/ExpressionBlockInputStream.cpp b/src/DataStreams/ExpressionBlockInputStream.cpp
deleted file mode 100644
index 4840a6263f6..00000000000
--- a/src/DataStreams/ExpressionBlockInputStream.cpp
+++ /dev/null
@@ -1,39 +0,0 @@
-#include
-#include
-
-
-namespace DB
-{
-
-ExpressionBlockInputStream::ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_)
- : expression(expression_)
-{
- children.push_back(input);
- cached_header = children.back()->getHeader();
- expression->execute(cached_header, true);
-}
-
-String ExpressionBlockInputStream::getName() const { return "Expression"; }
-
-Block ExpressionBlockInputStream::getTotals()
-{
- totals = children.back()->getTotals();
- expression->execute(totals);
-
- return totals;
-}
-
-Block ExpressionBlockInputStream::getHeader() const
-{
- return cached_header.cloneEmpty();
-}
-
-Block ExpressionBlockInputStream::readImpl()
-{
- Block res = children.back()->read();
- if (res)
- expression->execute(res);
- return res;
-}
-
-}
diff --git a/src/DataStreams/ExpressionBlockInputStream.h b/src/DataStreams/ExpressionBlockInputStream.h
deleted file mode 100644
index fae54fbcfbf..00000000000
--- a/src/DataStreams/ExpressionBlockInputStream.h
+++ /dev/null
@@ -1,52 +0,0 @@
-#pragma once
-
-#include
-
-
-namespace DB
-{
-
-class ExpressionActions;
-
-/** Executes a certain expression over the block.
- * The expression consists of column identifiers from the block, constants, common functions.
- * For example: hits * 2 + 3, url LIKE '%yandex%'
- * The expression processes each row independently of the others.
- */
-class ExpressionBlockInputStream : public IBlockInputStream
-{
-public:
- using ExpressionActionsPtr = std::shared_ptr;
-
- ExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_);
-
- String getName() const override;
- Block getTotals() override;
- Block getHeader() const override;
-
-protected:
- ExpressionActionsPtr expression;
-
- Block readImpl() override;
-
-private:
- Block cached_header;
-};
-
-/// ExpressionBlockInputStream that could generate many out blocks for single input block.
-class InflatingExpressionBlockInputStream : public ExpressionBlockInputStream
-{
-public:
- InflatingExpressionBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_)
- : ExpressionBlockInputStream(input, expression_)
- {}
-
-protected:
- Block readImpl() override;
-
-private:
- ExtraBlockPtr not_processed;
- size_t action_number = 0;
-};
-
-}
diff --git a/src/DataStreams/InternalTextLogsRowOutputStream.cpp b/src/DataStreams/InternalTextLogs.cpp
similarity index 94%
rename from src/DataStreams/InternalTextLogsRowOutputStream.cpp
rename to src/DataStreams/InternalTextLogs.cpp
index 14247b8f2aa..a5883d17f28 100644
--- a/src/DataStreams/InternalTextLogsRowOutputStream.cpp
+++ b/src/DataStreams/InternalTextLogs.cpp
@@ -1,4 +1,4 @@
-#include "InternalTextLogsRowOutputStream.h"
+#include "InternalTextLogs.h"
#include