mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
Merge branch 'master' into table-constraints
This commit is contained in:
commit
86fecc7501
2
.gitmodules
vendored
2
.gitmodules
vendored
@ -93,7 +93,7 @@
|
||||
url = https://github.com/ClickHouse-Extras/libunwind.git
|
||||
[submodule "contrib/simdjson"]
|
||||
path = contrib/simdjson
|
||||
url = https://github.com/ClickHouse-Extras/simdjson.git
|
||||
url = https://github.com/lemire/simdjson.git
|
||||
[submodule "contrib/rapidjson"]
|
||||
path = contrib/rapidjson
|
||||
url = https://github.com/Tencent/rapidjson
|
||||
|
@ -264,7 +264,9 @@ if (USE_STATIC_LIBRARIES AND HAVE_NO_PIE)
|
||||
set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${FLAG_NO_PIE}")
|
||||
endif ()
|
||||
|
||||
if (NOT SANITIZE AND NOT SPLIT_SHARED_LIBRARIES)
|
||||
# TODO: only make this extra-checks in CI builds, since a lot of contrib libs won't link -
|
||||
# CI works around this problem by explicitly adding GLIBC_COMPATIBILITY flag.
|
||||
if (NOT SANITIZE AND YANDEX_OFFICIAL_BUILD)
|
||||
set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -Wl,--no-undefined")
|
||||
endif ()
|
||||
@ -328,7 +330,7 @@ if (OS_LINUX AND NOT UNBUNDLED AND (GLIBC_COMPATIBILITY OR USE_INTERNAL_UNWIND_L
|
||||
if (USE_INTERNAL_LIBCXX_LIBRARY)
|
||||
set (LIBCXX_LIBS "${ClickHouse_BINARY_DIR}/contrib/libcxx-cmake/libcxx_static${${CMAKE_POSTFIX_VARIABLE}}.a ${ClickHouse_BINARY_DIR}/contrib/libcxxabi-cmake/libcxxabi_static${${CMAKE_POSTFIX_VARIABLE}}.a")
|
||||
else ()
|
||||
set (LIBCXX_LIBS "-lc++ -lc++abi")
|
||||
set (LIBCXX_LIBS "-lc++ -lc++abi -lc++fs")
|
||||
endif ()
|
||||
|
||||
set (DEFAULT_LIBS "${DEFAULT_LIBS} -Wl,-Bstatic ${LIBCXX_LIBS} ${EXCEPTION_HANDLING_LIBRARY} ${BUILTINS_LIB_PATH} -Wl,-Bdynamic")
|
||||
|
@ -13,7 +13,8 @@ ClickHouse is an open-source column-oriented database management system that all
|
||||
* You can also [fill this form](https://forms.yandex.com/surveys/meet-yandex-clickhouse-team/) to meet Yandex ClickHouse team in person.
|
||||
|
||||
## Upcoming Events
|
||||
* [ClickHouse Meetup in Mountain View](https://www.eventbrite.com/e/meetup-clickhouse-in-the-south-bay-registration-65935505873) on August 13.
|
||||
* [ClickHouse Meetup in Moscow](https://yandex.ru/promo/clickhouse/moscow-2019) on September 5.
|
||||
* [ClickHouse Meetup in Paris](https://www.eventbrite.com/e/clickhouse-paris-meetup-2019-registration-68493270215) on October 3.
|
||||
* [ClickHouse Meetup in Hong Kong](https://www.meetup.com/Hong-Kong-Machine-Learning-Meetup/events/263580542/) on October 17.
|
||||
* [ClickHouse Meetup in Shenzhen](https://www.huodongxing.com/event/3483759917300) on October 20.
|
||||
* [ClickHouse Meetup in Shanghai](https://www.huodongxing.com/event/4483760336000) on October 27.
|
||||
|
@ -1,8 +1,8 @@
|
||||
option(USE_INTERNAL_CPUINFO_LIBRARY "Set to FALSE to use system cpuinfo library instead of bundled" ${NOT_UNBUNDLED})
|
||||
|
||||
# Now we have no contrib/libcpuinfo, use from system.
|
||||
if (USE_INTERNAL_CPUINFO_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcpuinfo/include")
|
||||
#message (WARNING "submodule contrib/libcpuid is missing. to fix try run: \n git submodule update --init --recursive")
|
||||
if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcpuinfo/include")
|
||||
#message (WARNING "submodule contrib/libcpuinfo is missing. to fix try run: \n git submodule update --init --recursive")
|
||||
set (USE_INTERNAL_CPUINFO_LIBRARY 0)
|
||||
set (MISSING_INTERNAL_CPUINFO_LIBRARY 1)
|
||||
endif ()
|
||||
@ -12,7 +12,7 @@ if(NOT USE_INTERNAL_CPUINFO_LIBRARY)
|
||||
find_path(CPUINFO_INCLUDE_DIR NAMES cpuinfo.h PATHS ${CPUINFO_INCLUDE_PATHS})
|
||||
endif()
|
||||
|
||||
if(CPUID_LIBRARY AND CPUID_INCLUDE_DIR)
|
||||
if(CPUINFO_LIBRARY AND CPUINFO_INCLUDE_DIR)
|
||||
set(USE_CPUINFO 1)
|
||||
elseif(NOT MISSING_INTERNAL_CPUINFO_LIBRARY)
|
||||
set(CPUINFO_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcpuinfo/include)
|
||||
|
@ -3,8 +3,11 @@ if (NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include/simdjson/jsonp
|
||||
return()
|
||||
endif ()
|
||||
|
||||
if (NOT HAVE_AVX2)
|
||||
message (WARNING "submodule contrib/simdjson requires AVX2 support")
|
||||
if (NOT HAVE_SSE42)
|
||||
message (WARNING "submodule contrib/simdjson requires support of SSE4.2 instructions")
|
||||
return()
|
||||
elseif (NOT HAVE_PCLMULQDQ)
|
||||
message (WARNING "submodule contrib/simdjson requires support of PCLMULQDQ instructions")
|
||||
return()
|
||||
endif ()
|
||||
|
||||
|
@ -81,6 +81,17 @@ check_cxx_source_compiles("
|
||||
}
|
||||
" HAVE_AVX2)
|
||||
|
||||
set (TEST_FLAG "-mpclmul")
|
||||
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0")
|
||||
check_cxx_source_compiles("
|
||||
#include <wmmintrin.h>
|
||||
int main() {
|
||||
auto a = _mm_clmulepi64_si128(__m128i(), __m128i(), 0);
|
||||
(void)a;
|
||||
return 0;
|
||||
}
|
||||
" HAVE_PCLMULQDQ)
|
||||
|
||||
# gcc -dM -E -mpopcnt - < /dev/null | sort > gcc-dump-popcnt
|
||||
#define __POPCNT__ 1
|
||||
|
||||
|
2
contrib/mariadb-connector-c
vendored
2
contrib/mariadb-connector-c
vendored
@ -1 +1 @@
|
||||
Subproject commit d85d0e98999cd9e28ceb66645999b4a9ce85370e
|
||||
Subproject commit c6503d3acc85ca1a7f5e7e38b605d7c9410aac1e
|
@ -31,6 +31,7 @@ ${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_stmt_codec.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_string.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_time.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_tls.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/openssl_crypt.c
|
||||
#${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/gnutls.c
|
||||
#${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/ma_schannel.c
|
||||
#${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/schannel.c
|
||||
@ -42,6 +43,7 @@ ${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/mariadb_cleartext.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/my_auth.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/old_password.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sha256_pw.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/caching_sha2_pw.c
|
||||
#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sspi_client.c
|
||||
#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sspi_errmsg.c
|
||||
${MARIADB_CLIENT_SOURCE_DIR}/plugins/connection/aurora.c
|
||||
|
@ -76,17 +76,20 @@ struct st_client_plugin_int *plugin_list[MYSQL_CLIENT_MAX_PLUGINS + MARIADB_CLIE
|
||||
static pthread_mutex_t LOCK_load_client_plugin;
|
||||
#endif
|
||||
|
||||
extern struct st_mysql_client_plugin mysql_native_password_client_plugin;
|
||||
extern struct st_mysql_client_plugin mysql_old_password_client_plugin;
|
||||
extern struct st_mysql_client_plugin pvio_socket_client_plugin;
|
||||
extern struct st_mysql_client_plugin mysql_native_password_client_plugin;
|
||||
extern struct st_mysql_client_plugin mysql_old_password_client_plugin;
|
||||
extern struct st_mysql_client_plugin pvio_socket_client_plugin;
|
||||
extern struct st_mysql_client_plugin sha256_password_client_plugin;
|
||||
extern struct st_mysql_client_plugin caching_sha2_password_client_plugin;
|
||||
|
||||
|
||||
struct st_mysql_client_plugin *mysql_client_builtins[]=
|
||||
{
|
||||
(struct st_mysql_client_plugin *)&mysql_native_password_client_plugin,
|
||||
(struct st_mysql_client_plugin *)&mysql_old_password_client_plugin,
|
||||
(struct st_mysql_client_plugin *)&pvio_socket_client_plugin,
|
||||
|
||||
(struct st_mysql_client_plugin *)&mysql_native_password_client_plugin,
|
||||
(struct st_mysql_client_plugin *)&mysql_old_password_client_plugin,
|
||||
(struct st_mysql_client_plugin *)&pvio_socket_client_plugin,
|
||||
(struct st_mysql_client_plugin *)&sha256_password_client_plugin,
|
||||
(struct st_mysql_client_plugin *)&caching_sha2_password_client_plugin,
|
||||
0
|
||||
};
|
||||
|
||||
|
2
contrib/poco
vendored
2
contrib/poco
vendored
@ -1 +1 @@
|
||||
Subproject commit 7a2d304c21549427460428c9039009ef4bbfd899
|
||||
Subproject commit 6216cc01a107ce149863411ca29013a224f80343
|
2
contrib/simdjson
vendored
2
contrib/simdjson
vendored
@ -1 +1 @@
|
||||
Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355
|
||||
Subproject commit e9be643db5cf1c29a69bc80ee72d220124a9c50e
|
@ -1,6 +1,3 @@
|
||||
if (NOT HAVE_AVX2)
|
||||
message (FATAL_ERROR "No AVX2 support")
|
||||
endif ()
|
||||
set(SIMDJSON_INCLUDE_DIR "${ClickHouse_SOURCE_DIR}/contrib/simdjson/include")
|
||||
set(SIMDJSON_SRC_DIR "${SIMDJSON_INCLUDE_DIR}/../src")
|
||||
set(SIMDJSON_SRC
|
||||
@ -16,4 +13,3 @@ set(SIMDJSON_SRC
|
||||
|
||||
add_library(${SIMDJSON_LIBRARY} ${SIMDJSON_SRC})
|
||||
target_include_directories(${SIMDJSON_LIBRARY} SYSTEM PUBLIC "${SIMDJSON_INCLUDE_DIR}")
|
||||
target_compile_options(${SIMDJSON_LIBRARY} PRIVATE -mavx2 -mbmi -mbmi2 -mpclmul)
|
||||
|
@ -1,11 +1,11 @@
|
||||
# This strings autochanged from release_lib.sh:
|
||||
set(VERSION_REVISION 54425)
|
||||
set(VERSION_MAJOR 19)
|
||||
set(VERSION_MINOR 13)
|
||||
set(VERSION_MINOR 14)
|
||||
set(VERSION_PATCH 1)
|
||||
set(VERSION_GITHASH adfc36917222bdb03eba069f0cad0f4f5b8f1c94)
|
||||
set(VERSION_DESCRIBE v19.13.1.1-prestable)
|
||||
set(VERSION_STRING 19.13.1.1)
|
||||
set(VERSION_DESCRIBE v19.14.1.1-prestable)
|
||||
set(VERSION_STRING 19.14.1.1)
|
||||
# end of autochange
|
||||
|
||||
set(VERSION_EXTRA "" CACHE STRING "")
|
||||
|
@ -3,6 +3,8 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Common/CpuId.h>
|
||||
#include <common/getMemoryAmount.h>
|
||||
#include <DataStreams/copyData.h>
|
||||
#include <DataStreams/NullBlockOutputStream.h>
|
||||
#include <DataStreams/RemoteBlockInputStream.h>
|
||||
#include <IO/ConnectionTimeouts.h>
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
@ -249,6 +251,54 @@ std::vector<TestStats> PerformanceTest::execute()
|
||||
|
||||
runQueries(queries_with_indexes, statistics_by_run);
|
||||
}
|
||||
|
||||
if (got_SIGINT)
|
||||
{
|
||||
return statistics_by_run;
|
||||
}
|
||||
|
||||
// Pull memory usage data from query log. The log is normally filled in
|
||||
// background, so we have to flush it synchronously here to see all the
|
||||
// previous queries.
|
||||
{
|
||||
NullBlockOutputStream null_output(Block{});
|
||||
RemoteBlockInputStream flush_log(connection, "system flush logs",
|
||||
{} /* header */, context);
|
||||
copyData(flush_log, null_output);
|
||||
}
|
||||
|
||||
for (auto & statistics : statistics_by_run)
|
||||
{
|
||||
if (statistics.query_id.empty())
|
||||
{
|
||||
// We have statistics structs for skipped queries as well, so we
|
||||
// have to filter them out.
|
||||
continue;
|
||||
}
|
||||
|
||||
// We run some test queries several times, specifying the same query id,
|
||||
// so this query to the log may return several records. Choose the
|
||||
// last one, because this is when the query performance has stabilized.
|
||||
RemoteBlockInputStream log_reader(connection,
|
||||
"select memory_usage, query_start_time from system.query_log "
|
||||
"where type = 2 and query_id = '" + statistics.query_id + "' "
|
||||
"order by query_start_time desc",
|
||||
{} /* header */, context);
|
||||
|
||||
log_reader.readPrefix();
|
||||
Block block = log_reader.read();
|
||||
if (block.columns() == 0)
|
||||
{
|
||||
LOG_WARNING(log, "Query '" << statistics.query_id << "' is not found in query log.");
|
||||
continue;
|
||||
}
|
||||
|
||||
auto column = block.getByName("memory_usage").column;
|
||||
statistics.memory_usage = column->get64(0);
|
||||
|
||||
log_reader.readSuffix();
|
||||
}
|
||||
|
||||
return statistics_by_run;
|
||||
}
|
||||
|
||||
@ -298,47 +348,6 @@ void PerformanceTest::runQueries(
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (got_SIGINT)
|
||||
{
|
||||
return;
|
||||
}
|
||||
|
||||
// Pull memory usage data from query log. The log is normally filled in
|
||||
// background, so we have to flush it synchronously here to see all the
|
||||
// previous queries.
|
||||
{
|
||||
RemoteBlockInputStream flush_log(connection, "system flush logs",
|
||||
{} /* header */, context);
|
||||
flush_log.readPrefix();
|
||||
while (flush_log.read());
|
||||
flush_log.readSuffix();
|
||||
}
|
||||
|
||||
for (auto & statistics : statistics_by_run)
|
||||
{
|
||||
RemoteBlockInputStream log_reader(connection,
|
||||
"select memory_usage from system.query_log where type = 2 and query_id = '"
|
||||
+ statistics.query_id + "'",
|
||||
{} /* header */, context);
|
||||
|
||||
log_reader.readPrefix();
|
||||
Block block = log_reader.read();
|
||||
if (block.columns() == 0)
|
||||
{
|
||||
LOG_WARNING(log, "Query '" << statistics.query_id << "' is not found in query log.");
|
||||
continue;
|
||||
}
|
||||
|
||||
assert(block.columns() == 1);
|
||||
assert(block.getDataTypes()[0]->getName() == "UInt64");
|
||||
ColumnPtr column = block.getByPosition(0).column;
|
||||
assert(column->size() == 1);
|
||||
StringRef ref = column->getDataAt(0);
|
||||
assert(ref.size == sizeof(UInt64));
|
||||
statistics.memory_usage = *reinterpret_cast<const UInt64*>(ref.data);
|
||||
log_reader.readSuffix();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
|
@ -2,7 +2,6 @@
|
||||
|
||||
#include <limits>
|
||||
#include <ext/scope_guard.h>
|
||||
#include <openssl/rsa.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/config_version.h>
|
||||
#include <Common/NetException.h>
|
||||
@ -45,6 +44,7 @@ MySQLHandler::MySQLHandler(IServer & server_, const Poco::Net::StreamSocket & so
|
||||
, connection_id(connection_id_)
|
||||
, public_key(public_key_)
|
||||
, private_key(private_key_)
|
||||
, auth_plugin(new Authentication::Native41())
|
||||
{
|
||||
server_capability_flags = CLIENT_PROTOCOL_41 | CLIENT_SECURE_CONNECTION | CLIENT_PLUGIN_AUTH | CLIENT_PLUGIN_AUTH_LENENC_CLIENT_DATA | CLIENT_CONNECT_WITH_DB | CLIENT_DEPRECATE_EOF;
|
||||
if (ssl_enabled)
|
||||
@ -62,9 +62,7 @@ void MySQLHandler::run()
|
||||
|
||||
try
|
||||
{
|
||||
String scramble = generateScramble();
|
||||
|
||||
Handshake handshake(server_capability_flags, connection_id, VERSION_STRING + String("-") + VERSION_NAME, Authentication::Native, scramble + '\0');
|
||||
Handshake handshake(server_capability_flags, connection_id, VERSION_STRING + String("-") + VERSION_NAME, auth_plugin->getName(), auth_plugin->getAuthPluginData());
|
||||
packet_sender->sendPacket<Handshake>(handshake, true);
|
||||
|
||||
LOG_TRACE(log, "Sent handshake");
|
||||
@ -96,10 +94,21 @@ void MySQLHandler::run()
|
||||
client_capability_flags = handshake_response.capability_flags;
|
||||
if (!(client_capability_flags & CLIENT_PROTOCOL_41))
|
||||
throw Exception("Required capability: CLIENT_PROTOCOL_41.", ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES);
|
||||
if (!(client_capability_flags & CLIENT_PLUGIN_AUTH))
|
||||
throw Exception("Required capability: CLIENT_PLUGIN_AUTH.", ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES);
|
||||
|
||||
authenticate(handshake_response, scramble);
|
||||
authenticate(handshake_response.username, handshake_response.auth_plugin_name, handshake_response.auth_response);
|
||||
|
||||
try
|
||||
{
|
||||
if (!handshake_response.database.empty())
|
||||
connection_context.setCurrentDatabase(handshake_response.database);
|
||||
connection_context.setCurrentQueryId("");
|
||||
}
|
||||
catch (const Exception & exc)
|
||||
{
|
||||
log->log(exc);
|
||||
packet_sender->sendPacket(ERR_Packet(exc.code(), "00000", exc.message()), true);
|
||||
}
|
||||
|
||||
OK_Packet ok_packet(0, handshake_response.capability_flags, 0, 0, 0);
|
||||
packet_sender->sendPacket(ok_packet, true);
|
||||
|
||||
@ -216,121 +225,24 @@ void MySQLHandler::finishHandshake(MySQLProtocol::HandshakeResponse & packet)
|
||||
}
|
||||
}
|
||||
|
||||
String MySQLHandler::generateScramble()
|
||||
void MySQLHandler::authenticate(const String & user_name, const String & auth_plugin_name, const String & initial_auth_response)
|
||||
{
|
||||
String scramble(MySQLProtocol::SCRAMBLE_LENGTH, 0);
|
||||
Poco::RandomInputStream generator;
|
||||
for (size_t i = 0; i < scramble.size(); i++)
|
||||
{
|
||||
generator >> scramble[i];
|
||||
}
|
||||
return scramble;
|
||||
}
|
||||
// For compatibility with JavaScript MySQL client, Native41 authentication plugin is used when possible (if password is specified using double SHA1). Otherwise SHA256 plugin is used.
|
||||
auto user = connection_context.getUser(user_name);
|
||||
if (user->password_double_sha1_hex.empty())
|
||||
auth_plugin = std::make_unique<Authentication::Sha256Password>(public_key, private_key, log);
|
||||
|
||||
void MySQLHandler::authenticate(const HandshakeResponse & handshake_response, const String & scramble)
|
||||
{
|
||||
|
||||
String auth_response;
|
||||
AuthSwitchResponse response;
|
||||
if (handshake_response.auth_plugin_name != Authentication::SHA256)
|
||||
{
|
||||
/** Native authentication sent 20 bytes + '\0' character = 21 bytes.
|
||||
* This plugin must do the same to stay consistent with historical behavior if it is set to operate as a default plugin.
|
||||
* https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L3994
|
||||
*/
|
||||
packet_sender->sendPacket(AuthSwitchRequest(Authentication::SHA256, scramble + '\0'), true);
|
||||
if (in->eof())
|
||||
throw Exception(
|
||||
"Client doesn't support authentication method " + String(Authentication::SHA256) + " used by ClickHouse",
|
||||
ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES);
|
||||
packet_sender->receivePacket(response);
|
||||
auth_response = response.value;
|
||||
LOG_TRACE(log, "Authentication method mismatch.");
|
||||
}
|
||||
else
|
||||
{
|
||||
auth_response = handshake_response.auth_response;
|
||||
LOG_TRACE(log, "Authentication method match.");
|
||||
}
|
||||
|
||||
if (auth_response == "\1")
|
||||
{
|
||||
LOG_TRACE(log, "Client requests public key.");
|
||||
|
||||
BIO * mem = BIO_new(BIO_s_mem());
|
||||
SCOPE_EXIT(BIO_free(mem));
|
||||
if (PEM_write_bio_RSA_PUBKEY(mem, &public_key) != 1)
|
||||
{
|
||||
throw Exception("Failed to write public key to memory. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
|
||||
}
|
||||
char * pem_buf = nullptr;
|
||||
long pem_size = BIO_get_mem_data(mem, &pem_buf);
|
||||
String pem(pem_buf, pem_size);
|
||||
|
||||
LOG_TRACE(log, "Key: " << pem);
|
||||
|
||||
AuthMoreData data(pem);
|
||||
packet_sender->sendPacket(data, true);
|
||||
packet_sender->receivePacket(response);
|
||||
auth_response = response.value;
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(log, "Client didn't request public key.");
|
||||
}
|
||||
|
||||
String password;
|
||||
|
||||
/** Decrypt password, if it's not empty.
|
||||
* The original intention was that the password is a string[NUL] but this never got enforced properly so now we have to accept that
|
||||
* an empty packet is a blank password, thus the check for auth_response.empty() has to be made too.
|
||||
* https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L4017
|
||||
*/
|
||||
if (!secure_connection && !auth_response.empty() && auth_response != String("\0", 1))
|
||||
{
|
||||
LOG_TRACE(log, "Received nonempty password");
|
||||
auto ciphertext = reinterpret_cast<unsigned char *>(auth_response.data());
|
||||
|
||||
unsigned char plaintext[RSA_size(&private_key)];
|
||||
int plaintext_size = RSA_private_decrypt(auth_response.size(), ciphertext, plaintext, &private_key, RSA_PKCS1_OAEP_PADDING);
|
||||
if (plaintext_size == -1)
|
||||
{
|
||||
throw Exception("Failed to decrypt auth data. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
|
||||
}
|
||||
|
||||
password.resize(plaintext_size);
|
||||
for (int i = 0; i < plaintext_size; ++i)
|
||||
{
|
||||
password[i] = plaintext[i] ^ static_cast<unsigned char>(scramble[i % scramble.size()]);
|
||||
}
|
||||
}
|
||||
else if (secure_connection)
|
||||
{
|
||||
password = auth_response;
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(log, "Received empty password");
|
||||
}
|
||||
|
||||
if (!password.empty() && password.back() == 0)
|
||||
{
|
||||
password.pop_back();
|
||||
}
|
||||
|
||||
try
|
||||
{
|
||||
connection_context.setUser(handshake_response.username, password, socket().address(), "");
|
||||
if (!handshake_response.database.empty()) connection_context.setCurrentDatabase(handshake_response.database);
|
||||
connection_context.setCurrentQueryId("");
|
||||
LOG_INFO(log, "Authentication for user " << handshake_response.username << " succeeded.");
|
||||
try {
|
||||
std::optional<String> auth_response = auth_plugin_name == auth_plugin->getName() ? std::make_optional<String>(initial_auth_response) : std::nullopt;
|
||||
auth_plugin->authenticate(user_name, auth_response, connection_context, packet_sender, secure_connection, socket().address());
|
||||
}
|
||||
catch (const Exception & exc)
|
||||
{
|
||||
LOG_ERROR(log, "Authentication for user " << handshake_response.username << " failed.");
|
||||
LOG_ERROR(log, "Authentication for user " << user_name << " failed.");
|
||||
packet_sender->sendPacket(ERR_Packet(exc.code(), "00000", exc.message()), true);
|
||||
throw;
|
||||
}
|
||||
LOG_INFO(log, "Authentication for user " << user_name << " succeeded.");
|
||||
}
|
||||
|
||||
void MySQLHandler::comInitDB(ReadBuffer & payload)
|
||||
|
@ -30,9 +30,7 @@ private:
|
||||
|
||||
void comInitDB(ReadBuffer & payload);
|
||||
|
||||
static String generateScramble();
|
||||
|
||||
void authenticate(const MySQLProtocol::HandshakeResponse &, const String & scramble);
|
||||
void authenticate(const String & user_name, const String & auth_plugin_name, const String & auth_response);
|
||||
|
||||
IServer & server;
|
||||
Poco::Logger * log;
|
||||
@ -48,6 +46,8 @@ private:
|
||||
RSA & public_key;
|
||||
RSA & private_key;
|
||||
|
||||
std::unique_ptr<MySQLProtocol::Authentication::IPlugin> auth_plugin;
|
||||
|
||||
std::shared_ptr<Poco::Net::SecureStreamSocket> ss;
|
||||
std::shared_ptr<ReadBuffer> in;
|
||||
std::shared_ptr<WriteBuffer> out;
|
||||
|
@ -29,6 +29,7 @@
|
||||
#include <Common/getFQDNOrHostName.h>
|
||||
#include <Common/getMultipleKeysFromConfig.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/getExecutablePath.h>
|
||||
#include <Common/TaskStatsInfoGetter.h>
|
||||
#include <Common/ThreadStatus.h>
|
||||
#include <IO/HTTPCommon.h>
|
||||
@ -212,6 +213,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
const auto memory_amount = getMemoryAmount();
|
||||
|
||||
#if defined(__linux__)
|
||||
std::string executable_path = getExecutablePath();
|
||||
if (executable_path.empty())
|
||||
executable_path = "/usr/bin/clickhouse"; /// It is used for information messages.
|
||||
|
||||
/// After full config loaded
|
||||
{
|
||||
if (config().getBool("mlock_executable", false))
|
||||
@ -228,7 +233,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
LOG_INFO(log, "It looks like the process has no CAP_IPC_LOCK capability, binary mlock will be disabled."
|
||||
" It could happen due to incorrect ClickHouse package installation."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep /usr/bin/clickhouse'."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_ipc_lock=+ep " << executable_path << "'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems.");
|
||||
}
|
||||
}
|
||||
@ -547,7 +552,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
LOG_INFO(log, "It looks like the process has no CAP_NET_ADMIN capability, 'taskstats' performance statistics will be disabled."
|
||||
" It could happen due to incorrect ClickHouse package installation."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_net_admin=+ep /usr/bin/clickhouse'."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_net_admin=+ep " << executable_path << "'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems."
|
||||
" It also doesn't work if you run clickhouse-server inside network namespace as it happens in some containers.");
|
||||
}
|
||||
@ -556,7 +561,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
|
||||
{
|
||||
LOG_INFO(log, "It looks like the process has no CAP_SYS_NICE capability, the setting 'os_thread_nice' will have no effect."
|
||||
" It could happen due to incorrect ClickHouse package installation."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep /usr/bin/clickhouse'."
|
||||
" You could resolve the problem manually with 'sudo setcap cap_sys_nice=+ep " << executable_path << "'."
|
||||
" Note that it will not work on 'nosuid' mounted filesystems.");
|
||||
}
|
||||
#else
|
||||
|
8
dbms/programs/server/config.d/metric_log.xml
Normal file
8
dbms/programs/server/config.d/metric_log.xml
Normal file
@ -0,0 +1,8 @@
|
||||
<yandex>
|
||||
<metric_log>
|
||||
<database>system</database>
|
||||
<table>metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
</metric_log>
|
||||
</yandex>
|
@ -332,6 +332,16 @@
|
||||
</text_log>
|
||||
-->
|
||||
|
||||
<!-- Uncomment to write metric log into table.
|
||||
Metric log contains rows with current values of ProfileEvents, CurrentMetrics collected with "collect_interval_milliseconds" interval.
|
||||
<metric_log>
|
||||
<database>system</database>
|
||||
<table>metric_log</table>
|
||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||
<collect_interval_milliseconds>1000</collect_interval_milliseconds>
|
||||
</metric_log>
|
||||
-->
|
||||
|
||||
<!-- Parameters for embedded dictionaries, used in Yandex.Metrica.
|
||||
See https://clickhouse.yandex/docs/en/dicts/internal_dicts/
|
||||
-->
|
||||
|
@ -39,10 +39,18 @@
|
||||
|
||||
If you want to specify SHA256, place it in 'password_sha256_hex' element.
|
||||
Example: <password_sha256_hex>65e84be33532fb784c48129675f9eff3a682b27168c0ea744b2cf58ee02337c5</password_sha256_hex>
|
||||
Restrictions of SHA256: impossibility to connect to ClickHouse using MySQL JS client (as of July 2019).
|
||||
|
||||
If you want to specify double SHA1, place it in 'password_double_sha1_hex' element.
|
||||
Example: <password_double_sha1_hex>e395796d6546b1b65db9d665cd43f0e858dd4303</password_double_sha1_hex>
|
||||
|
||||
How to generate decent password:
|
||||
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-'
|
||||
In first line will be password and in second - corresponding SHA256.
|
||||
|
||||
How to generate double SHA1:
|
||||
Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | openssl dgst -sha1 -binary | openssl dgst -sha1
|
||||
In first line will be password and in second - corresponding double SHA1.
|
||||
-->
|
||||
<password></password>
|
||||
|
||||
|
@ -62,12 +62,6 @@ public:
|
||||
static_cast<ColumnUInt64 &>(to).getData().push_back(data(place).count);
|
||||
}
|
||||
|
||||
/// May be used for optimization.
|
||||
void addDelta(AggregateDataPtr place, UInt64 x) const
|
||||
{
|
||||
data(place).count += x;
|
||||
}
|
||||
|
||||
const char * getHeaderFilePath() const override { return __FILE__; }
|
||||
};
|
||||
|
||||
|
@ -128,6 +128,15 @@ public:
|
||||
using AddFunc = void (*)(const IAggregateFunction *, AggregateDataPtr, const IColumn **, size_t, Arena *);
|
||||
virtual AddFunc getAddressOfAddFunction() const = 0;
|
||||
|
||||
/** Contains a loop with calls to "add" function. You can collect arguments into array "places"
|
||||
* and do a single call to "addBatch" for devirtualization and inlining.
|
||||
*/
|
||||
virtual void addBatch(size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, Arena * arena) const = 0;
|
||||
|
||||
/** The same for single place.
|
||||
*/
|
||||
virtual void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0;
|
||||
|
||||
/** This is used for runtime code generation to determine, which header files to include in generated source.
|
||||
* Always implement it as
|
||||
* const char * getHeaderFilePath() const override { return __FILE__; }
|
||||
@ -156,7 +165,20 @@ private:
|
||||
public:
|
||||
IAggregateFunctionHelper(const DataTypes & argument_types_, const Array & parameters_)
|
||||
: IAggregateFunction(argument_types_, parameters_) {}
|
||||
|
||||
AddFunc getAddressOfAddFunction() const override { return &addFree; }
|
||||
|
||||
void addBatch(size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, Arena * arena) const override
|
||||
{
|
||||
for (size_t i = 0; i < batch_size; ++i)
|
||||
static_cast<const Derived *>(this)->add(places[i] + place_offset, columns, i, arena);
|
||||
}
|
||||
|
||||
void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override
|
||||
{
|
||||
for (size_t i = 0; i < batch_size; ++i)
|
||||
static_cast<const Derived *>(this)->add(place, columns, i, arena);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -26,6 +26,11 @@ void CurrentThread::updatePerformanceCounters()
|
||||
current_thread->updatePerformanceCounters();
|
||||
}
|
||||
|
||||
bool CurrentThread::isInitialized()
|
||||
{
|
||||
return current_thread;
|
||||
}
|
||||
|
||||
ThreadStatus & CurrentThread::get()
|
||||
{
|
||||
if (unlikely(!current_thread))
|
||||
|
@ -33,6 +33,9 @@ class InternalTextLogsQueue;
|
||||
class CurrentThread
|
||||
{
|
||||
public:
|
||||
/// Return true in case of successful initializaiton
|
||||
static bool isInitialized();
|
||||
|
||||
/// Handler to current thread
|
||||
static ThreadStatus & get();
|
||||
|
||||
|
@ -172,7 +172,7 @@
|
||||
M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \
|
||||
M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \
|
||||
\
|
||||
M(QueryProfilerCannotWriteTrace, "Number of stack traces dropped by query profiler because pipe is full or cannot write to pipe.") \
|
||||
M(CannotWriteToWriteBufferDiscard, "Number of stack traces dropped by query profiler or signal handler because pipe is full or cannot write to pipe.") \
|
||||
M(QueryProfilerSignalOverruns, "Number of times we drop processing of a signal due to overrun plus the number of signals that OS has not delivered due to overrun.") \
|
||||
|
||||
namespace ProfileEvents
|
||||
|
@ -11,12 +11,11 @@
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/thread_local_rng.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
|
||||
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event QueryProfilerCannotWriteTrace;
|
||||
extern const Event QueryProfilerSignalOverruns;
|
||||
}
|
||||
|
||||
@ -27,36 +26,6 @@ extern LazyPipe trace_pipe;
|
||||
|
||||
namespace
|
||||
{
|
||||
/** Write to file descriptor but drop the data if write would block or fail.
|
||||
* To use within signal handler. Motivating example: a signal handler invoked during execution of malloc
|
||||
* should not block because some mutex (or even worse - a spinlock) may be held.
|
||||
*/
|
||||
class WriteBufferDiscardOnFailure : public WriteBufferFromFileDescriptor
|
||||
{
|
||||
protected:
|
||||
void nextImpl() override
|
||||
{
|
||||
size_t bytes_written = 0;
|
||||
while (bytes_written != offset())
|
||||
{
|
||||
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
|
||||
|
||||
if ((-1 == res || 0 == res) && errno != EINTR)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::QueryProfilerCannotWriteTrace);
|
||||
break; /// Discard
|
||||
}
|
||||
|
||||
if (res > 0)
|
||||
bytes_written += res;
|
||||
}
|
||||
}
|
||||
|
||||
public:
|
||||
using WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor;
|
||||
~WriteBufferDiscardOnFailure() override {}
|
||||
};
|
||||
|
||||
/// Normally query_id is a UUID (string with a fixed length) but user can provide custom query_id.
|
||||
/// Thus upper bound on query_id length should be introduced to avoid buffer overflow in signal handler.
|
||||
constexpr size_t QUERY_ID_MAX_LEN = 1024;
|
||||
@ -90,7 +59,7 @@ namespace
|
||||
sizeof(TimerType) + // timer type
|
||||
sizeof(UInt32); // thread_number
|
||||
char buffer[buf_size];
|
||||
WriteBufferDiscardOnFailure out(trace_pipe.fds_rw[1], buf_size, buffer);
|
||||
WriteBufferFromFileDescriptorDiscardOnFailure out(trace_pipe.fds_rw[1], buf_size, buffer);
|
||||
|
||||
StringRef query_id = CurrentThread::getQueryId();
|
||||
query_id.size = std::min(query_id.size, QUERY_ID_MAX_LEN);
|
||||
@ -204,11 +173,13 @@ QueryProfilerBase<ProfilerImpl>::~QueryProfilerBase()
|
||||
template <typename ProfilerImpl>
|
||||
void QueryProfilerBase<ProfilerImpl>::tryCleanup()
|
||||
{
|
||||
#if USE_INTERNAL_UNWIND_LIBRARY
|
||||
if (timer_id != nullptr && timer_delete(timer_id))
|
||||
LOG_ERROR(log, "Failed to delete query profiler timer " + errnoToString(ErrorCodes::CANNOT_DELETE_TIMER));
|
||||
|
||||
if (previous_handler != nullptr && sigaction(pause_signal, previous_handler, nullptr))
|
||||
LOG_ERROR(log, "Failed to restore signal handler after query profiler " + errnoToString(ErrorCodes::CANNOT_SET_SIGNAL_HANDLER));
|
||||
#endif
|
||||
}
|
||||
|
||||
template class QueryProfilerBase<QueryProfilerReal>;
|
||||
|
@ -1,7 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
|
||||
#include <common/config_common.h>
|
||||
#include <signal.h>
|
||||
#include <time.h>
|
||||
|
||||
@ -43,8 +43,10 @@ private:
|
||||
|
||||
Poco::Logger * log;
|
||||
|
||||
#if USE_INTERNAL_UNWIND_LIBRARY
|
||||
/// Timer id from timer_create(2)
|
||||
timer_t timer_id = nullptr;
|
||||
#endif
|
||||
|
||||
/// Pause signal to interrupt threads to get traces
|
||||
int pause_signal;
|
||||
|
@ -151,6 +151,12 @@ std::string signalToErrorMessage(int sig, const siginfo_t & info, const ucontext
|
||||
}
|
||||
break;
|
||||
}
|
||||
|
||||
case SIGPROF:
|
||||
{
|
||||
error << "This is a signal used for debugging purposes by the user.";
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return error.str();
|
||||
@ -239,10 +245,10 @@ const StackTrace::Frames & StackTrace::getFrames() const
|
||||
}
|
||||
|
||||
|
||||
static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset, size_t size)
|
||||
static void toStringEveryLineImpl(const StackTrace::Frames & frames, size_t offset, size_t size, std::function<void(const std::string &)> callback)
|
||||
{
|
||||
if (size == 0)
|
||||
return "<Empty trace>";
|
||||
return callback("<Empty trace>");
|
||||
|
||||
const DB::SymbolIndex & symbol_index = DB::SymbolIndex::instance();
|
||||
std::unordered_map<std::string, DB::Dwarf> dwarfs;
|
||||
@ -281,12 +287,23 @@ static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset
|
||||
else
|
||||
out << "?";
|
||||
|
||||
out << "\n";
|
||||
callback(out.str());
|
||||
out.str({});
|
||||
}
|
||||
}
|
||||
|
||||
static std::string toStringImpl(const StackTrace::Frames & frames, size_t offset, size_t size)
|
||||
{
|
||||
std::stringstream out;
|
||||
toStringEveryLineImpl(frames, offset, size, [&](const std::string & str) { out << str << '\n'; });
|
||||
return out.str();
|
||||
}
|
||||
|
||||
void StackTrace::toStringEveryLine(std::function<void(const std::string &)> callback) const
|
||||
{
|
||||
toStringEveryLineImpl(frames, offset, size, std::move(callback));
|
||||
}
|
||||
|
||||
std::string StackTrace::toString() const
|
||||
{
|
||||
/// Calculation of stack trace text is extremely slow.
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <vector>
|
||||
#include <array>
|
||||
#include <optional>
|
||||
#include <functional>
|
||||
#include <signal.h>
|
||||
|
||||
#ifdef __APPLE__
|
||||
@ -39,6 +40,8 @@ public:
|
||||
const Frames & getFrames() const;
|
||||
std::string toString() const;
|
||||
|
||||
void toStringEveryLine(std::function<void(const std::string &)> callback) const;
|
||||
|
||||
protected:
|
||||
void tryCapture();
|
||||
|
||||
|
13
dbms/src/Common/getExecutablePath.cpp
Normal file
13
dbms/src/Common/getExecutablePath.cpp
Normal file
@ -0,0 +1,13 @@
|
||||
#include <Common/getExecutablePath.h>
|
||||
#include <filesystem>
|
||||
|
||||
|
||||
std::string getExecutablePath()
|
||||
{
|
||||
std::error_code ec;
|
||||
std::filesystem::path canonical_path = std::filesystem::canonical("/proc/self/exe", ec);
|
||||
|
||||
if (ec)
|
||||
return {};
|
||||
return canonical_path;
|
||||
}
|
11
dbms/src/Common/getExecutablePath.h
Normal file
11
dbms/src/Common/getExecutablePath.h
Normal file
@ -0,0 +1,11 @@
|
||||
#pragma once
|
||||
|
||||
#include <string>
|
||||
|
||||
/** Get path to the running executable if possible.
|
||||
* It is possible when:
|
||||
* - procfs exists;
|
||||
* - there is a /proc/self/exe file;
|
||||
* Otherwise return empty string.
|
||||
*/
|
||||
std::string getExecutablePath();
|
@ -1,3 +1,4 @@
|
||||
#include <malloc.h>
|
||||
#include <new>
|
||||
|
||||
#include <common/config_common.h>
|
||||
@ -49,6 +50,11 @@ ALWAYS_INLINE void untrackMemory(void * ptr [[maybe_unused]], std::size_t size [
|
||||
#else
|
||||
if (size)
|
||||
CurrentMemoryTracker::free(size);
|
||||
#ifdef _GNU_SOURCE
|
||||
/// It's innaccurate resource free for sanitizers. malloc_usable_size() result is greater or equal to allocated size.
|
||||
else
|
||||
CurrentMemoryTracker::free(malloc_usable_size(ptr));
|
||||
#endif
|
||||
#endif
|
||||
}
|
||||
catch (...)
|
||||
|
@ -36,7 +36,7 @@ target_include_directories (simple_cache PRIVATE ${DBMS_INCLUDE_DIR})
|
||||
target_link_libraries (simple_cache PRIVATE common)
|
||||
|
||||
add_executable (compact_array compact_array.cpp)
|
||||
target_link_libraries (compact_array PRIVATE clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY})
|
||||
target_link_libraries (compact_array PRIVATE clickhouse_common_io stdc++fs)
|
||||
|
||||
add_executable (radix_sort radix_sort.cpp)
|
||||
target_link_libraries (radix_sort PRIVATE clickhouse_common_io)
|
||||
|
@ -78,11 +78,12 @@ binary_value_info getLeadingAndTrailingBits(const T & value)
|
||||
const UInt8 lz = getLeadingZeroBits(value);
|
||||
const UInt8 tz = getTrailingZeroBits(value);
|
||||
const UInt8 data_size = value == 0 ? 0 : static_cast<UInt8>(bit_size - lz - tz);
|
||||
|
||||
return binary_value_info{lz, data_size, tz};
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
|
||||
UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest, UInt32 dest_size)
|
||||
{
|
||||
static const auto DATA_BIT_LENGTH = getBitLengthOfLength(sizeof(T));
|
||||
// -1 since there must be at least 1 non-zero bit.
|
||||
@ -91,6 +92,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
|
||||
if (source_size % sizeof(T) != 0)
|
||||
throw Exception("Cannot compress, data size " + toString(source_size) + " is not aligned to " + toString(sizeof(T)), ErrorCodes::CANNOT_COMPRESS);
|
||||
const char * source_end = source + source_size;
|
||||
const char * dest_end = dest + dest_size;
|
||||
|
||||
const UInt32 items_count = source_size / sizeof(T);
|
||||
|
||||
@ -110,7 +112,7 @@ UInt32 compressDataForType(const char * source, UInt32 source_size, char * dest)
|
||||
dest += sizeof(prev_value);
|
||||
}
|
||||
|
||||
WriteBuffer buffer(dest, getCompressedDataSize(sizeof(T), source_size - sizeof(items_count) - sizeof(prev_value)));
|
||||
WriteBuffer buffer(dest, dest_end - dest);
|
||||
BitWriter writer(buffer);
|
||||
|
||||
while (source < source_end)
|
||||
@ -265,24 +267,26 @@ UInt32 CompressionCodecGorilla::doCompressData(const char * source, UInt32 sourc
|
||||
dest[1] = bytes_to_skip;
|
||||
memcpy(&dest[2], source, bytes_to_skip);
|
||||
size_t start_pos = 2 + bytes_to_skip;
|
||||
UInt32 compressed_size = 0;
|
||||
UInt32 result_size = 0;
|
||||
|
||||
const UInt32 compressed_size = getMaxCompressedDataSize(source_size);
|
||||
switch (data_bytes_size)
|
||||
{
|
||||
case 1:
|
||||
compressed_size = compressDataForType<UInt8>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
result_size = compressDataForType<UInt8>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos], compressed_size);
|
||||
break;
|
||||
case 2:
|
||||
compressed_size = compressDataForType<UInt16>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
result_size = compressDataForType<UInt16>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos], compressed_size);
|
||||
break;
|
||||
case 4:
|
||||
compressed_size = compressDataForType<UInt32>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
result_size = compressDataForType<UInt32>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos], compressed_size);
|
||||
break;
|
||||
case 8:
|
||||
compressed_size = compressDataForType<UInt64>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos]);
|
||||
result_size = compressDataForType<UInt64>(&source[bytes_to_skip], source_size - bytes_to_skip, &dest[start_pos], compressed_size);
|
||||
break;
|
||||
}
|
||||
|
||||
return 1 + 1 + compressed_size;
|
||||
return 1 + 1 + result_size;
|
||||
}
|
||||
|
||||
void CompressionCodecGorilla::doDecompressData(const char * source, UInt32 source_size, char * dest, UInt32 /* uncompressed_size */) const
|
||||
|
@ -49,8 +49,8 @@ UInt32 ICompressionCodec::decompress(const char * source, UInt32 source_size, ch
|
||||
UInt8 header_size = getHeaderSize();
|
||||
UInt32 decompressed_size = unalignedLoad<UInt32>(&source[5]);
|
||||
doDecompressData(&source[header_size], source_size - header_size, dest, decompressed_size);
|
||||
return decompressed_size;
|
||||
|
||||
return decompressed_size;
|
||||
}
|
||||
|
||||
UInt32 ICompressionCodec::readCompressedBlockSize(const char * source)
|
||||
|
@ -1,10 +1,14 @@
|
||||
#include <Compression/CompressionCodecDoubleDelta.h>
|
||||
#include <Compression/CompressionCodecGorilla.h>
|
||||
#include <Compression/CompressionFactory.h>
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Parsers/ExpressionElementParsers.h>
|
||||
#include <Parsers/IParser.h>
|
||||
#include <Parsers/TokenIterator.h>
|
||||
|
||||
#include <boost/format.hpp>
|
||||
|
||||
@ -20,10 +24,44 @@
|
||||
|
||||
#include <string.h>
|
||||
|
||||
/// For the expansion of gtest macros.
|
||||
#if defined(__clang__)
|
||||
#pragma clang diagnostic ignored "-Wdeprecated"
|
||||
#elif defined (__GNUC__) && __GNUC__ >= 9
|
||||
#pragma GCC diagnostic ignored "-Wdeprecated-copy"
|
||||
#endif
|
||||
|
||||
#include <gtest/gtest.h>
|
||||
|
||||
using namespace DB;
|
||||
|
||||
namespace std
|
||||
{
|
||||
template <typename T>
|
||||
std::ostream & operator<<(std::ostream & ostr, const std::optional<T> & opt)
|
||||
{
|
||||
if (!opt)
|
||||
{
|
||||
return ostr << "<empty optional>";
|
||||
}
|
||||
|
||||
return ostr << *opt;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
std::vector<T> operator+(std::vector<T> && left, std::vector<T> && right)
|
||||
{
|
||||
std::vector<T> result(std::move(left));
|
||||
std::move(std::begin(right), std::end(right), std::back_inserter(result));
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
std::string bin(const T & value, size_t bits = sizeof(T)*8)
|
||||
{
|
||||
@ -37,43 +75,46 @@ std::string bin(const T & value, size_t bits = sizeof(T)*8)
|
||||
template <typename T>
|
||||
const char* type_name()
|
||||
{
|
||||
#define MAKE_TYPE_NAME(TYPE) \
|
||||
if constexpr (std::is_same_v<TYPE, T>) return #TYPE
|
||||
|
||||
MAKE_TYPE_NAME(UInt8);
|
||||
MAKE_TYPE_NAME(UInt16);
|
||||
MAKE_TYPE_NAME(UInt32);
|
||||
MAKE_TYPE_NAME(UInt64);
|
||||
MAKE_TYPE_NAME(Int8);
|
||||
MAKE_TYPE_NAME(Int16);
|
||||
MAKE_TYPE_NAME(Int32);
|
||||
MAKE_TYPE_NAME(Int64);
|
||||
MAKE_TYPE_NAME(Float32);
|
||||
MAKE_TYPE_NAME(Float64);
|
||||
|
||||
#undef MAKE_TYPE_NAME
|
||||
|
||||
return typeid(T).name();
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* type_name<UInt32>()
|
||||
template <typename T>
|
||||
DataTypePtr makeDataType()
|
||||
{
|
||||
return "uint32";
|
||||
}
|
||||
#define MAKE_DATA_TYPE(TYPE) \
|
||||
if constexpr (std::is_same_v<T, TYPE>) return std::make_shared<DataType ## TYPE>()
|
||||
|
||||
template <>
|
||||
const char* type_name<Int32>()
|
||||
{
|
||||
return "int32";
|
||||
}
|
||||
MAKE_DATA_TYPE(UInt8);
|
||||
MAKE_DATA_TYPE(UInt16);
|
||||
MAKE_DATA_TYPE(UInt32);
|
||||
MAKE_DATA_TYPE(UInt64);
|
||||
MAKE_DATA_TYPE(Int8);
|
||||
MAKE_DATA_TYPE(Int16);
|
||||
MAKE_DATA_TYPE(Int32);
|
||||
MAKE_DATA_TYPE(Int64);
|
||||
MAKE_DATA_TYPE(Float32);
|
||||
MAKE_DATA_TYPE(Float64);
|
||||
|
||||
template <>
|
||||
const char* type_name<UInt64>()
|
||||
{
|
||||
return "uint64";
|
||||
}
|
||||
#undef MAKE_DATA_TYPE
|
||||
|
||||
template <>
|
||||
const char* type_name<Int64>()
|
||||
{
|
||||
return "int64";
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* type_name<Float32>()
|
||||
{
|
||||
return "float";
|
||||
}
|
||||
|
||||
template <>
|
||||
const char* type_name<Float64>()
|
||||
{
|
||||
return "double";
|
||||
assert(false && "unsupported size");
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
|
||||
@ -135,52 +176,100 @@ template <typename T, typename ContainerLeft, typename ContainerRight>
|
||||
return result;
|
||||
}
|
||||
|
||||
struct CodecTestParam
|
||||
struct Codec
|
||||
{
|
||||
std::string type_name;
|
||||
std::vector<char> source_data;
|
||||
UInt8 data_byte_size;
|
||||
double min_compression_ratio;
|
||||
std::string case_name;
|
||||
std::string codec_statement;
|
||||
std::optional<double> expected_compression_ratio;
|
||||
|
||||
// to allow setting ratio after building with complex builder functions.
|
||||
CodecTestParam && setRatio(const double & ratio) &&
|
||||
{
|
||||
this->min_compression_ratio = ratio;
|
||||
return std::move(*this);
|
||||
}
|
||||
explicit Codec(std::string codec_statement_, std::optional<double> expected_compression_ratio_ = std::nullopt)
|
||||
: codec_statement(std::move(codec_statement_)),
|
||||
expected_compression_ratio(expected_compression_ratio_)
|
||||
{}
|
||||
|
||||
Codec()
|
||||
: Codec(std::string())
|
||||
{}
|
||||
};
|
||||
|
||||
CodecTestParam operator+(CodecTestParam && left, CodecTestParam && right)
|
||||
|
||||
struct CodecTestSequence
|
||||
{
|
||||
assert(left.type_name == right.type_name);
|
||||
assert(left.data_byte_size == right.data_byte_size);
|
||||
std::string name;
|
||||
std::vector<char> serialized_data;
|
||||
DataTypePtr data_type;
|
||||
|
||||
std::vector data(std::move(left.source_data));
|
||||
data.insert(data.end(), right.source_data.begin(), right.source_data.end());
|
||||
CodecTestSequence()
|
||||
: name(),
|
||||
serialized_data(),
|
||||
data_type()
|
||||
{}
|
||||
|
||||
return CodecTestParam{
|
||||
left.type_name,
|
||||
std::move(data),
|
||||
left.data_byte_size,
|
||||
std::min(left.min_compression_ratio, right.min_compression_ratio),
|
||||
left.case_name + " + " + right.case_name
|
||||
CodecTestSequence(std::string name_, std::vector<char> serialized_data_, DataTypePtr data_type_)
|
||||
: name(name_),
|
||||
serialized_data(serialized_data_),
|
||||
data_type(data_type_)
|
||||
{}
|
||||
|
||||
CodecTestSequence(const CodecTestSequence &) = default;
|
||||
CodecTestSequence & operator=(const CodecTestSequence &) = default;
|
||||
CodecTestSequence(CodecTestSequence &&) = default;
|
||||
CodecTestSequence & operator=(CodecTestSequence &&) = default;
|
||||
};
|
||||
|
||||
CodecTestSequence operator+(CodecTestSequence && left, CodecTestSequence && right)
|
||||
{
|
||||
assert(left.data_type->equals(*right.data_type));
|
||||
|
||||
std::vector<char> data(std::move(left.serialized_data));
|
||||
data.insert(data.end(), right.serialized_data.begin(), right.serialized_data.end());
|
||||
|
||||
return CodecTestSequence{
|
||||
left.name + " + " + right.name,
|
||||
std::move(data),
|
||||
std::move(left.data_type)
|
||||
};
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & ostr, const CodecTestParam & param)
|
||||
template <typename T>
|
||||
CodecTestSequence operator*(CodecTestSequence && left, T times)
|
||||
{
|
||||
return ostr << "name: " << param.case_name
|
||||
<< "\ntype name:" << param.type_name
|
||||
<< "\nbyte size: " << static_cast<UInt32>(param.data_byte_size)
|
||||
<< "\ndata size: " << param.source_data.size();
|
||||
std::vector<char> data(std::move(left.serialized_data));
|
||||
const size_t initial_size = data.size();
|
||||
const size_t final_size = initial_size * times;
|
||||
|
||||
data.reserve(final_size);
|
||||
|
||||
for (T i = 0; i < times; ++i)
|
||||
{
|
||||
data.insert(data.end(), data.begin(), data.begin() + initial_size);
|
||||
}
|
||||
|
||||
return CodecTestSequence{
|
||||
left.name + " x " + std::to_string(times),
|
||||
std::move(data),
|
||||
std::move(left.data_type)
|
||||
};
|
||||
}
|
||||
|
||||
// compression ratio < 1.0 means that codec output is smaller than input.
|
||||
const double DEFAULT_MIN_COMPRESSION_RATIO = 1.0;
|
||||
std::ostream & operator<<(std::ostream & ostr, const Codec & codec)
|
||||
{
|
||||
return ostr << "Codec{"
|
||||
<< "name: " << codec.codec_statement
|
||||
<< ", expected_compression_ratio: " << codec.expected_compression_ratio
|
||||
<< "}";
|
||||
}
|
||||
|
||||
std::ostream & operator<<(std::ostream & ostr, const CodecTestSequence & seq)
|
||||
{
|
||||
return ostr << "CodecTestSequence{"
|
||||
<< "name: " << seq.name
|
||||
<< ", type name: " << seq.data_type->getName()
|
||||
<< ", data size: " << seq.serialized_data.size() << " bytes"
|
||||
<< "}";
|
||||
}
|
||||
|
||||
template <typename T, typename... Args>
|
||||
CodecTestParam makeParam(Args && ... args)
|
||||
CodecTestSequence makeSeq(Args && ... args)
|
||||
{
|
||||
std::initializer_list<T> vals{static_cast<T>(args)...};
|
||||
std::vector<char> data(sizeof(T) * std::size(vals));
|
||||
@ -192,14 +281,17 @@ CodecTestParam makeParam(Args && ... args)
|
||||
write_pos += sizeof(v);
|
||||
}
|
||||
|
||||
return CodecTestParam{type_name<T>(), std::move(data), sizeof(T), DEFAULT_MIN_COMPRESSION_RATIO,
|
||||
(boost::format("%1% values of %2%") % std::size(vals) % type_name<T>()).str()};
|
||||
return CodecTestSequence{
|
||||
(boost::format("%1% values of %2%") % std::size(vals) % type_name<T>()).str(),
|
||||
std::move(data),
|
||||
makeDataType<T>()
|
||||
};
|
||||
}
|
||||
|
||||
template <typename T, size_t Begin = 1, size_t End = 10001, typename Generator>
|
||||
CodecTestParam generateParam(Generator gen, const char* gen_name)
|
||||
template <typename T, typename Generator>
|
||||
CodecTestSequence generateSeq(Generator gen, const char* gen_name, size_t Begin = 0, size_t End = 10000)
|
||||
{
|
||||
static_assert (End >= Begin, "End must be not less than Begin");
|
||||
assert (End >= Begin);
|
||||
|
||||
std::vector<char> data(sizeof(T) * (End - Begin));
|
||||
char * write_pos = data.data();
|
||||
@ -211,89 +303,104 @@ CodecTestParam generateParam(Generator gen, const char* gen_name)
|
||||
write_pos += sizeof(v);
|
||||
}
|
||||
|
||||
return CodecTestParam{type_name<T>(), std::move(data), sizeof(T), DEFAULT_MIN_COMPRESSION_RATIO,
|
||||
(boost::format("%1% values of %2% from %3%") % (End - Begin) % type_name<T>() % gen_name).str()};
|
||||
return CodecTestSequence{
|
||||
(boost::format("%1% values of %2% from %3%") % (End - Begin) % type_name<T>() % gen_name).str(),
|
||||
std::move(data),
|
||||
makeDataType<T>()
|
||||
};
|
||||
}
|
||||
|
||||
void TestTranscoding(ICompressionCodec * codec, const CodecTestParam & param)
|
||||
{
|
||||
const auto & source_data = param.source_data;
|
||||
|
||||
const UInt32 encoded_max_size = codec->getCompressedReserveSize(source_data.size());
|
||||
PODArray<char> encoded(encoded_max_size);
|
||||
|
||||
const UInt32 encoded_size = codec->compress(source_data.data(), source_data.size(), encoded.data());
|
||||
encoded.resize(encoded_size);
|
||||
|
||||
PODArray<char> decoded(source_data.size());
|
||||
const UInt32 decoded_size = codec->decompress(encoded.data(), encoded.size(), decoded.data());
|
||||
decoded.resize(decoded_size);
|
||||
|
||||
switch (param.data_byte_size)
|
||||
{
|
||||
case 1:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt8>(source_data, decoded));
|
||||
break;
|
||||
case 2:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt16>(source_data, decoded));
|
||||
break;
|
||||
case 4:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt32>(source_data, decoded));
|
||||
break;
|
||||
case 8:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt64>(source_data, decoded));
|
||||
break;
|
||||
default:
|
||||
FAIL() << "Invalid data_byte_size: " << param.data_byte_size;
|
||||
}
|
||||
const auto header_size = codec->getHeaderSize();
|
||||
const auto compression_ratio = (encoded_size - header_size) / (source_data.size() * 1.0);
|
||||
|
||||
ASSERT_LE(compression_ratio, param.min_compression_ratio)
|
||||
<< "\n\tdecoded size: " << source_data.size()
|
||||
<< "\n\tencoded size: " << encoded_size
|
||||
<< "(no header: " << encoded_size - header_size << ")";
|
||||
}
|
||||
|
||||
class CodecTest : public ::testing::TestWithParam<CodecTestParam>
|
||||
class CodecTest : public ::testing::TestWithParam<std::tuple<Codec, CodecTestSequence>>
|
||||
{
|
||||
public:
|
||||
static void SetUpTestCase()
|
||||
enum MakeCodecParam
|
||||
{
|
||||
// To make random predicatble and avoid failing test "out of the blue".
|
||||
srand(0);
|
||||
CODEC_WITH_DATA_TYPE,
|
||||
CODEC_WITHOUT_DATA_TYPE,
|
||||
};
|
||||
|
||||
CompressionCodecPtr makeCodec(MakeCodecParam with_data_type) const
|
||||
{
|
||||
const auto & codec_string = std::get<0>(GetParam()).codec_statement;
|
||||
const auto & data_type = with_data_type == CODEC_WITH_DATA_TYPE ? std::get<1>(GetParam()).data_type : nullptr;
|
||||
|
||||
const std::string codec_statement = "(" + codec_string + ")";
|
||||
Tokens tokens(codec_statement.begin().base(), codec_statement.end().base());
|
||||
IParser::Pos token_iterator(tokens);
|
||||
|
||||
Expected expected;
|
||||
ASTPtr codec_ast;
|
||||
ParserCodec parser;
|
||||
|
||||
parser.parse(token_iterator, codec_ast, expected);
|
||||
|
||||
return CompressionCodecFactory::instance().get(codec_ast, data_type);
|
||||
}
|
||||
|
||||
void testTranscoding(ICompressionCodec & codec)
|
||||
{
|
||||
const auto & test_sequence = std::get<1>(GetParam());
|
||||
const auto & source_data = test_sequence.serialized_data;
|
||||
|
||||
const UInt32 encoded_max_size = codec.getCompressedReserveSize(source_data.size());
|
||||
PODArray<char> encoded(encoded_max_size);
|
||||
|
||||
const UInt32 encoded_size = codec.compress(source_data.data(), source_data.size(), encoded.data());
|
||||
encoded.resize(encoded_size);
|
||||
|
||||
PODArray<char> decoded(source_data.size());
|
||||
const UInt32 decoded_size = codec.decompress(encoded.data(), encoded.size(), decoded.data());
|
||||
decoded.resize(decoded_size);
|
||||
|
||||
switch (test_sequence.data_type->getSizeOfValueInMemory())
|
||||
{
|
||||
case 1:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt8>(source_data, decoded));
|
||||
break;
|
||||
case 2:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt16>(source_data, decoded));
|
||||
break;
|
||||
case 4:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt32>(source_data, decoded));
|
||||
break;
|
||||
case 8:
|
||||
ASSERT_TRUE(EqualByteContainersAs<UInt64>(source_data, decoded));
|
||||
break;
|
||||
default:
|
||||
FAIL() << "Invalid test sequence data type: " << test_sequence.data_type->getName();
|
||||
}
|
||||
const auto header_size = codec.getHeaderSize();
|
||||
const auto compression_ratio = (encoded_size - header_size) / (source_data.size() * 1.0);
|
||||
|
||||
const auto & codec_spec = std::get<0>(GetParam());
|
||||
if (codec_spec.expected_compression_ratio)
|
||||
{
|
||||
ASSERT_LE(compression_ratio, *codec_spec.expected_compression_ratio)
|
||||
<< "\n\tdecoded size: " << source_data.size()
|
||||
<< "\n\tencoded size: " << encoded_size
|
||||
<< "(no header: " << encoded_size - header_size << ")";
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
TEST_P(CodecTest, DoubleDelta)
|
||||
TEST_P(CodecTest, TranscodingWithDataType)
|
||||
{
|
||||
auto param = GetParam();
|
||||
auto codec = std::make_unique<CompressionCodecDoubleDelta>(param.data_byte_size);
|
||||
if (param.type_name == type_name<Float32>() || param.type_name == type_name<Float64>())
|
||||
{
|
||||
// dd doesn't work great with many cases of integers and may result in very poor compression rate.
|
||||
param.min_compression_ratio *= 1.5;
|
||||
}
|
||||
|
||||
TestTranscoding(codec.get(), param);
|
||||
const auto codec = makeCodec(CODEC_WITH_DATA_TYPE);
|
||||
testTranscoding(*codec);
|
||||
}
|
||||
|
||||
TEST_P(CodecTest, Gorilla)
|
||||
TEST_P(CodecTest, TranscodingWithoutDataType)
|
||||
{
|
||||
auto param = GetParam();
|
||||
auto codec = std::make_unique<CompressionCodecGorilla>(param.data_byte_size);
|
||||
if (param.type_name == type_name<UInt32>() || param.type_name == type_name<Int32>()
|
||||
|| param.type_name == type_name<UInt64>() || param.type_name == type_name<Int64>())
|
||||
{
|
||||
// gorilla doesn't work great with many cases of integers and may result in very poor compression rate.
|
||||
param.min_compression_ratio *= 1.5;
|
||||
}
|
||||
|
||||
TestTranscoding(codec.get(), param);
|
||||
const auto codec = makeCodec(CODEC_WITHOUT_DATA_TYPE);
|
||||
testTranscoding(*codec);
|
||||
}
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
// Here we use generators to produce test payload for codecs.
|
||||
// Generator is a callable that should produce output value of the same type as input value.
|
||||
// Generator is a callable that can produce infinite number of values,
|
||||
// output value MUST be of the same type input value.
|
||||
///////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
auto SameValueGenerator = [](auto value)
|
||||
{
|
||||
@ -332,141 +439,427 @@ auto SequentialGenerator = [](auto stride = 1)
|
||||
//};
|
||||
|
||||
template <typename T>
|
||||
using uniform_distribution =
|
||||
typename std::conditional_t<std::is_floating_point_v<T>, std::uniform_real_distribution<T>,
|
||||
typename std::conditional_t<std::is_integral_v<T>, std::uniform_int_distribution<T>, void>>;
|
||||
|
||||
|
||||
template <typename T = Int32>
|
||||
struct MonotonicGenerator
|
||||
{
|
||||
MonotonicGenerator(T stride_ = 1, size_t max_step_ = 10)
|
||||
MonotonicGenerator(T stride_ = 1, T max_step = 10)
|
||||
: prev_value(0),
|
||||
stride(stride_),
|
||||
max_step(max_step_)
|
||||
random_engine(0),
|
||||
distribution(0, max_step)
|
||||
{}
|
||||
|
||||
template <typename U>
|
||||
U operator()(U)
|
||||
{
|
||||
const U result = prev_value + static_cast<T>(stride * (rand() % max_step));
|
||||
|
||||
prev_value = result;
|
||||
return result;
|
||||
prev_value = prev_value + stride * distribution(random_engine);
|
||||
return static_cast<U>(prev_value);
|
||||
}
|
||||
|
||||
private:
|
||||
T prev_value;
|
||||
const T stride;
|
||||
const size_t max_step;
|
||||
};
|
||||
|
||||
auto MinMaxGenerator = [](auto i)
|
||||
{
|
||||
if (i % 2 == 0)
|
||||
{
|
||||
return std::numeric_limits<decltype(i)>::min();
|
||||
}
|
||||
else
|
||||
{
|
||||
return std::numeric_limits<decltype(i)>::max();
|
||||
}
|
||||
std::default_random_engine random_engine;
|
||||
uniform_distribution<T> distribution;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
struct RandomGenerator
|
||||
{
|
||||
RandomGenerator(T seed = 0, T value_cap_ = std::numeric_limits<T>::max())
|
||||
: e(seed),
|
||||
value_cap(value_cap_)
|
||||
RandomGenerator(T seed = 0, T value_min = std::numeric_limits<T>::min(), T value_max = std::numeric_limits<T>::max())
|
||||
: random_engine(seed),
|
||||
distribution(value_min, value_max)
|
||||
{
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
U operator()(U i)
|
||||
U operator()(U)
|
||||
{
|
||||
return static_cast<decltype(i)>(distribution(e) % value_cap);
|
||||
return static_cast<U>(distribution(random_engine));
|
||||
}
|
||||
|
||||
private:
|
||||
std::default_random_engine e;
|
||||
std::uniform_int_distribution<T> distribution;
|
||||
const T value_cap;
|
||||
std::default_random_engine random_engine;
|
||||
uniform_distribution<T> distribution;
|
||||
};
|
||||
|
||||
auto RandomishGenerator = [](auto i)
|
||||
{
|
||||
return static_cast<decltype(i)>(sin(static_cast<double>(i) * i) * i);
|
||||
return static_cast<decltype(i)>(sin(static_cast<double>(i * i)) * i);
|
||||
};
|
||||
|
||||
// helper macro to produce human-friendly test case name
|
||||
auto MinMaxGenerator = []()
|
||||
{
|
||||
return [step = 0](auto i) mutable
|
||||
{
|
||||
if (step++ % 2 == 0)
|
||||
{
|
||||
return std::numeric_limits<decltype(i)>::min();
|
||||
}
|
||||
else
|
||||
{
|
||||
return std::numeric_limits<decltype(i)>::max();
|
||||
}
|
||||
};
|
||||
};
|
||||
|
||||
// Fill dest value with 0x00 or 0xFF
|
||||
auto FFand0Generator = []()
|
||||
{
|
||||
return [step = 0](auto i) mutable
|
||||
{
|
||||
decltype(i) result;
|
||||
if (step++ % 2 == 0)
|
||||
{
|
||||
memset(&result, 0, sizeof(result));
|
||||
}
|
||||
else
|
||||
{
|
||||
memset(&result, 0xFF, sizeof(result));
|
||||
}
|
||||
|
||||
return result;
|
||||
};
|
||||
};
|
||||
|
||||
|
||||
// Makes many sequences with generator, first sequence length is 1, second is 2... up to `sequences_count`.
|
||||
template <typename T, typename Generator>
|
||||
std::vector<CodecTestSequence> generatePyramidOfSequences(const size_t sequences_count, Generator && generator, const char* generator_name)
|
||||
{
|
||||
std::vector<CodecTestSequence> sequences;
|
||||
sequences.reserve(sequences_count);
|
||||
for (size_t i = 1; i < sequences_count; ++i)
|
||||
{
|
||||
std::string name = generator_name + std::string(" from 0 to ") + std::to_string(i);
|
||||
sequences.push_back(generateSeq<T>(std::forward<decltype(generator)>(generator), name.c_str(), 0, i));
|
||||
}
|
||||
|
||||
return sequences;
|
||||
};
|
||||
|
||||
|
||||
// helper macro to produce human-friendly sequence name from generator
|
||||
#define G(generator) generator, #generator
|
||||
|
||||
const auto DefaultCodecsToTest = ::testing::Values(
|
||||
Codec("DoubleDelta"),
|
||||
Codec("DoubleDelta, LZ4"),
|
||||
Codec("DoubleDelta, ZSTD"),
|
||||
Codec("Gorilla"),
|
||||
Codec("Gorilla, LZ4"),
|
||||
Codec("Gorilla, ZSTD")
|
||||
);
|
||||
|
||||
///////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
// test cases
|
||||
///////////////////////////////////////////////////////////////////////////////////////////////////
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Simple,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
makeSeq<Float64>(1, 2, 3, 5, 7, 11, 13, 17, 23, 29, 31, 37, 41, 43, 47, 53, 59, 61, 67, 71, 73, 79, 83, 89, 97)
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(SmallSequences,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::ValuesIn(
|
||||
generatePyramidOfSequences<Int8 >(42, G(SequentialGenerator(1)))
|
||||
+ generatePyramidOfSequences<Int16 >(42, G(SequentialGenerator(1)))
|
||||
+ generatePyramidOfSequences<Int32 >(42, G(SequentialGenerator(1)))
|
||||
+ generatePyramidOfSequences<Int64 >(42, G(SequentialGenerator(1)))
|
||||
+ generatePyramidOfSequences<UInt8 >(42, G(SequentialGenerator(1)))
|
||||
+ generatePyramidOfSequences<UInt16>(42, G(SequentialGenerator(1)))
|
||||
+ generatePyramidOfSequences<UInt32>(42, G(SequentialGenerator(1)))
|
||||
+ generatePyramidOfSequences<UInt64>(42, G(SequentialGenerator(1)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Mixed,
|
||||
CodecTest,
|
||||
::testing::Values(
|
||||
generateParam<Int32, 1, 3>(G(MinMaxGenerator)) + generateParam<Int32, 1, 11>(G(SequentialGenerator(1))).setRatio(1),
|
||||
generateParam<UInt32, 1, 3>(G(MinMaxGenerator)) + generateParam<UInt32, 1, 11>(G(SequentialGenerator(1))).setRatio(1),
|
||||
generateParam<Int64, 1, 3>(G(MinMaxGenerator)) + generateParam<Int64, 1, 11>(G(SequentialGenerator(1))).setRatio(1),
|
||||
generateParam<UInt64, 1, 3>(G(MinMaxGenerator)) + generateParam<UInt64, 1, 11>(G(SequentialGenerator(1))).setRatio(1)
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int8>(G(MinMaxGenerator()), 1, 5) + generateSeq<Int8>(G(SequentialGenerator(1)), 1, 1001),
|
||||
generateSeq<Int16>(G(MinMaxGenerator()), 1, 5) + generateSeq<Int16>(G(SequentialGenerator(1)), 1, 1001),
|
||||
generateSeq<Int32>(G(MinMaxGenerator()), 1, 5) + generateSeq<Int32>(G(SequentialGenerator(1)), 1, 1001),
|
||||
generateSeq<Int64>(G(MinMaxGenerator()), 1, 5) + generateSeq<Int64>(G(SequentialGenerator(1)), 1, 1001),
|
||||
generateSeq<UInt8>(G(MinMaxGenerator()), 1, 5) + generateSeq<UInt8>(G(SequentialGenerator(1)), 1, 1001),
|
||||
generateSeq<UInt16>(G(MinMaxGenerator()), 1, 5) + generateSeq<UInt16>(G(SequentialGenerator(1)), 1, 1001),
|
||||
generateSeq<UInt32>(G(MinMaxGenerator()), 1, 5) + generateSeq<UInt32>(G(SequentialGenerator(1)), 1, 1001),
|
||||
generateSeq<UInt64>(G(MinMaxGenerator()), 1, 5) + generateSeq<UInt64>(G(SequentialGenerator(1)), 1, 1001)
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Same,
|
||||
INSTANTIATE_TEST_CASE_P(SameValueInt,
|
||||
CodecTest,
|
||||
::testing::Values(
|
||||
generateParam<UInt32>(G(SameValueGenerator(1000))),
|
||||
generateParam<Int32>(G(SameValueGenerator(-1000))),
|
||||
generateParam<UInt64>(G(SameValueGenerator(1000))),
|
||||
generateParam<Int64>(G(SameValueGenerator(-1000))),
|
||||
generateParam<Float32>(G(SameValueGenerator(M_E))),
|
||||
generateParam<Float64>(G(SameValueGenerator(M_E)))
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int8 >(G(SameValueGenerator(1000))),
|
||||
generateSeq<Int16 >(G(SameValueGenerator(1000))),
|
||||
generateSeq<Int32 >(G(SameValueGenerator(1000))),
|
||||
generateSeq<Int64 >(G(SameValueGenerator(1000))),
|
||||
generateSeq<UInt8 >(G(SameValueGenerator(1000))),
|
||||
generateSeq<UInt16>(G(SameValueGenerator(1000))),
|
||||
generateSeq<UInt32>(G(SameValueGenerator(1000))),
|
||||
generateSeq<UInt64>(G(SameValueGenerator(1000)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Sequential,
|
||||
INSTANTIATE_TEST_CASE_P(SameNegativeValueInt,
|
||||
CodecTest,
|
||||
::testing::Values(
|
||||
generateParam<UInt32>(G(SequentialGenerator(1))),
|
||||
generateParam<Int32>(G(SequentialGenerator(-1))),
|
||||
generateParam<UInt64>(G(SequentialGenerator(1))),
|
||||
generateParam<Int64>(G(SequentialGenerator(-1))),
|
||||
generateParam<Float32>(G(SequentialGenerator(M_E))),
|
||||
generateParam<Float64>(G(SequentialGenerator(M_E)))
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int8 >(G(SameValueGenerator(-1000))),
|
||||
generateSeq<Int16 >(G(SameValueGenerator(-1000))),
|
||||
generateSeq<Int32 >(G(SameValueGenerator(-1000))),
|
||||
generateSeq<Int64 >(G(SameValueGenerator(-1000))),
|
||||
generateSeq<UInt8 >(G(SameValueGenerator(-1000))),
|
||||
generateSeq<UInt16>(G(SameValueGenerator(-1000))),
|
||||
generateSeq<UInt32>(G(SameValueGenerator(-1000))),
|
||||
generateSeq<UInt64>(G(SameValueGenerator(-1000)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Monotonic,
|
||||
INSTANTIATE_TEST_CASE_P(SameValueFloat,
|
||||
CodecTest,
|
||||
::testing::Values(
|
||||
generateParam<UInt32>(G(MonotonicGenerator<UInt32>(1, 5))),
|
||||
generateParam<Int32>(G(MonotonicGenerator<Int32>(-1, 5))),
|
||||
generateParam<UInt64>(G(MonotonicGenerator<UInt64>(1, 5))),
|
||||
generateParam<Int64>(G(MonotonicGenerator<Int64>(-1, 5))),
|
||||
generateParam<Float32>(G(MonotonicGenerator<Float32>(M_E, 5))),
|
||||
generateParam<Float64>(G(MonotonicGenerator<Float64>(M_E, 5)))
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("Gorilla"),
|
||||
Codec("Gorilla, LZ4")
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(SameValueGenerator(M_E))),
|
||||
generateSeq<Float64>(G(SameValueGenerator(M_E)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Random,
|
||||
INSTANTIATE_TEST_CASE_P(SameNegativeValueFloat,
|
||||
CodecTest,
|
||||
::testing::Values(
|
||||
generateParam<UInt32>(G(RandomGenerator<UInt32>(0, 1000'000'000))).setRatio(1.2),
|
||||
generateParam<UInt64>(G(RandomGenerator<UInt64>(0, 1000'000'000))).setRatio(1.1)
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("Gorilla"),
|
||||
Codec("Gorilla, LZ4")
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(SameValueGenerator(-1 * M_E))),
|
||||
generateSeq<Float64>(G(SameValueGenerator(-1 * M_E)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Randomish,
|
||||
INSTANTIATE_TEST_CASE_P(SequentialInt,
|
||||
CodecTest,
|
||||
::testing::Values(
|
||||
generateParam<Int32>(G(RandomishGenerator)).setRatio(1.1),
|
||||
generateParam<Int64>(G(RandomishGenerator)).setRatio(1.1),
|
||||
generateParam<UInt32>(G(RandomishGenerator)).setRatio(1.1),
|
||||
generateParam<UInt64>(G(RandomishGenerator)).setRatio(1.1),
|
||||
generateParam<Float32>(G(RandomishGenerator)).setRatio(1.1),
|
||||
generateParam<Float64>(G(RandomishGenerator)).setRatio(1.1)
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int8 >(G(SequentialGenerator(1))),
|
||||
generateSeq<Int16 >(G(SequentialGenerator(1))),
|
||||
generateSeq<Int32 >(G(SequentialGenerator(1))),
|
||||
generateSeq<Int64 >(G(SequentialGenerator(1))),
|
||||
generateSeq<UInt8 >(G(SequentialGenerator(1))),
|
||||
generateSeq<UInt16>(G(SequentialGenerator(1))),
|
||||
generateSeq<UInt32>(G(SequentialGenerator(1))),
|
||||
generateSeq<UInt64>(G(SequentialGenerator(1)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(Overflow,
|
||||
// -1, -2, -3, ... etc for signed
|
||||
// 0xFF, 0xFE, 0xFD, ... for unsigned
|
||||
INSTANTIATE_TEST_CASE_P(SequentialReverseInt,
|
||||
CodecTest,
|
||||
::testing::Values(
|
||||
generateParam<UInt32>(G(MinMaxGenerator)),
|
||||
generateParam<Int32>(G(MinMaxGenerator)),
|
||||
generateParam<UInt64>(G(MinMaxGenerator)),
|
||||
generateParam<Int64>(G(MinMaxGenerator))
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int8 >(G(SequentialGenerator(-1))),
|
||||
generateSeq<Int16 >(G(SequentialGenerator(-1))),
|
||||
generateSeq<Int32 >(G(SequentialGenerator(-1))),
|
||||
generateSeq<Int64 >(G(SequentialGenerator(-1))),
|
||||
generateSeq<UInt8 >(G(SequentialGenerator(-1))),
|
||||
generateSeq<UInt16>(G(SequentialGenerator(-1))),
|
||||
generateSeq<UInt32>(G(SequentialGenerator(-1))),
|
||||
generateSeq<UInt64>(G(SequentialGenerator(-1)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(SequentialFloat,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("Gorilla"),
|
||||
Codec("Gorilla, LZ4")
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(SequentialGenerator(M_E))),
|
||||
generateSeq<Float64>(G(SequentialGenerator(M_E)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(SequentialReverseFloat,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("Gorilla"),
|
||||
Codec("Gorilla, LZ4")
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(SequentialGenerator(-1 * M_E))),
|
||||
generateSeq<Float64>(G(SequentialGenerator(-1 * M_E)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(MonotonicInt,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int8 >(G(MonotonicGenerator(1, 5))),
|
||||
generateSeq<Int16 >(G(MonotonicGenerator(1, 5))),
|
||||
generateSeq<Int32 >(G(MonotonicGenerator(1, 5))),
|
||||
generateSeq<Int64 >(G(MonotonicGenerator(1, 5))),
|
||||
generateSeq<UInt8 >(G(MonotonicGenerator(1, 5))),
|
||||
generateSeq<UInt16>(G(MonotonicGenerator(1, 5))),
|
||||
generateSeq<UInt32>(G(MonotonicGenerator(1, 5))),
|
||||
generateSeq<UInt64>(G(MonotonicGenerator(1, 5)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(MonotonicReverseInt,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int8 >(G(MonotonicGenerator(-1, 5))),
|
||||
generateSeq<Int16 >(G(MonotonicGenerator(-1, 5))),
|
||||
generateSeq<Int32 >(G(MonotonicGenerator(-1, 5))),
|
||||
generateSeq<Int64 >(G(MonotonicGenerator(-1, 5))),
|
||||
generateSeq<UInt8 >(G(MonotonicGenerator(-1, 5))),
|
||||
generateSeq<UInt16>(G(MonotonicGenerator(-1, 5))),
|
||||
generateSeq<UInt32>(G(MonotonicGenerator(-1, 5))),
|
||||
generateSeq<UInt64>(G(MonotonicGenerator(-1, 5)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(MonotonicFloat,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("Gorilla")
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(MonotonicGenerator<Float32>(M_E, 5))),
|
||||
generateSeq<Float64>(G(MonotonicGenerator<Float64>(M_E, 5)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(MonotonicReverseFloat,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("Gorilla")
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(MonotonicGenerator<Float32>(-1 * M_E, 5))),
|
||||
generateSeq<Float64>(G(MonotonicGenerator<Float64>(-1 * M_E, 5)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(RandomInt,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<UInt8 >(G(RandomGenerator<UInt8>(0))),
|
||||
generateSeq<UInt16>(G(RandomGenerator<UInt16>(0))),
|
||||
generateSeq<UInt32>(G(RandomGenerator<UInt32>(0, 0, 1000'000'000))),
|
||||
generateSeq<UInt64>(G(RandomGenerator<UInt64>(0, 0, 1000'000'000)))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(RandomishInt,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Int32>(G(RandomishGenerator)),
|
||||
generateSeq<Int64>(G(RandomishGenerator)),
|
||||
generateSeq<UInt32>(G(RandomishGenerator)),
|
||||
generateSeq<UInt64>(G(RandomishGenerator)),
|
||||
generateSeq<Float32>(G(RandomishGenerator)),
|
||||
generateSeq<Float64>(G(RandomishGenerator))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(RandomishFloat,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
DefaultCodecsToTest,
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(RandomishGenerator)),
|
||||
generateSeq<Float64>(G(RandomishGenerator))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
// Double delta overflow case, deltas are out of bounds for target type
|
||||
INSTANTIATE_TEST_CASE_P(OverflowInt,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("DoubleDelta", 1.2),
|
||||
Codec("DoubleDelta, LZ4", 1.0)
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<UInt32>(G(MinMaxGenerator())),
|
||||
generateSeq<Int32>(G(MinMaxGenerator())),
|
||||
generateSeq<UInt64>(G(MinMaxGenerator())),
|
||||
generateSeq<Int64>(G(MinMaxGenerator()))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
INSTANTIATE_TEST_CASE_P(OverflowFloat,
|
||||
CodecTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(
|
||||
Codec("Gorilla", 1.1),
|
||||
Codec("Gorilla, LZ4", 1.0)
|
||||
),
|
||||
::testing::Values(
|
||||
generateSeq<Float32>(G(MinMaxGenerator())),
|
||||
generateSeq<Float64>(G(MinMaxGenerator())),
|
||||
generateSeq<Float32>(G(FFand0Generator())),
|
||||
generateSeq<Float64>(G(FFand0Generator()))
|
||||
)
|
||||
),
|
||||
);
|
||||
|
||||
}
|
||||
|
@ -92,6 +92,7 @@
|
||||
#endif
|
||||
|
||||
/// Check for presence of address sanitizer
|
||||
#if !defined(ADDRESS_SANITIZER)
|
||||
#if defined(__has_feature)
|
||||
#if __has_feature(address_sanitizer)
|
||||
#define ADDRESS_SANITIZER 1
|
||||
@ -99,7 +100,9 @@
|
||||
#elif defined(__SANITIZE_ADDRESS__)
|
||||
#define ADDRESS_SANITIZER 1
|
||||
#endif
|
||||
#endif
|
||||
|
||||
#if !defined(THREAD_SANITIZER)
|
||||
#if defined(__has_feature)
|
||||
#if __has_feature(thread_sanitizer)
|
||||
#define THREAD_SANITIZER 1
|
||||
@ -107,7 +110,9 @@
|
||||
#elif defined(__SANITIZE_THREAD__)
|
||||
#define THREAD_SANITIZER 1
|
||||
#endif
|
||||
#endif
|
||||
|
||||
#if !defined(MEMORY_SANITIZER)
|
||||
#if defined(__has_feature)
|
||||
#if __has_feature(memory_sanitizer)
|
||||
#define MEMORY_SANITIZER 1
|
||||
@ -115,6 +120,7 @@
|
||||
#elif defined(__MEMORY_SANITIZER__)
|
||||
#define MEMORY_SANITIZER 1
|
||||
#endif
|
||||
#endif
|
||||
|
||||
/// Explicitly allow undefined behaviour for certain functions. Use it as a function attribute.
|
||||
/// It is useful in case when compiler cannot see (and exploit) it, but UBSan can.
|
||||
|
@ -1,9 +1,17 @@
|
||||
#pragma once
|
||||
|
||||
#include <ext/scope_guard.h>
|
||||
#include <openssl/pem.h>
|
||||
#include <openssl/rsa.h>
|
||||
#include <random>
|
||||
#include <sstream>
|
||||
#include <Common/MemoryTracker.h>
|
||||
#include <Common/OpenSSLHelpers.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Core/Types.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/copyData.h>
|
||||
#include <IO/LimitReadBuffer.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/ReadBufferFromPocoSocket.h>
|
||||
@ -14,9 +22,7 @@
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Poco/Net/StreamSocket.h>
|
||||
#include <Poco/RandomStream.h>
|
||||
#include <random>
|
||||
#include <sstream>
|
||||
#include <IO/LimitReadBuffer.h>
|
||||
#include <Poco/SHA1Engine.h>
|
||||
|
||||
/// Implementation of MySQL wire protocol.
|
||||
/// Works only on little-endian architecture.
|
||||
@ -27,6 +33,9 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_PACKET_FROM_CLIENT;
|
||||
extern const int MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES;
|
||||
extern const int OPENSSL_ERROR;
|
||||
extern const int UNKNOWN_EXCEPTION;
|
||||
}
|
||||
|
||||
namespace MySQLProtocol
|
||||
@ -39,11 +48,6 @@ const size_t MYSQL_ERRMSG_SIZE = 512;
|
||||
const size_t PACKET_HEADER_SIZE = 4;
|
||||
const size_t SSL_REQUEST_PAYLOAD_SIZE = 32;
|
||||
|
||||
namespace Authentication
|
||||
{
|
||||
const String Native = "mysql_native_password";
|
||||
const String SHA256 = "sha256_password"; /// Caching SHA2 plugin is not used because it would be possible to authenticate knowing hash from users.xml.
|
||||
}
|
||||
|
||||
enum CharacterSet
|
||||
{
|
||||
@ -149,6 +153,8 @@ private:
|
||||
uint8_t & sequence_id;
|
||||
const size_t max_packet_size = MAX_PACKET_LENGTH;
|
||||
|
||||
bool has_read_header = false;
|
||||
|
||||
// Size of packet which is being read now.
|
||||
size_t payload_length = 0;
|
||||
|
||||
@ -158,8 +164,9 @@ private:
|
||||
protected:
|
||||
bool nextImpl() override
|
||||
{
|
||||
if (payload_length == 0 || (payload_length == max_packet_size && offset == payload_length))
|
||||
if (!has_read_header || (payload_length == max_packet_size && offset == payload_length))
|
||||
{
|
||||
has_read_header = true;
|
||||
working_buffer.resize(0);
|
||||
offset = 0;
|
||||
payload_length = 0;
|
||||
@ -171,10 +178,6 @@ protected:
|
||||
tmp << "Received packet with payload larger than max_packet_size: " << payload_length;
|
||||
throw ProtocolError(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT);
|
||||
}
|
||||
else if (payload_length == 0)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
size_t packet_sequence_id = 0;
|
||||
in.read(reinterpret_cast<char &>(packet_sequence_id));
|
||||
@ -185,6 +188,9 @@ protected:
|
||||
throw ProtocolError(tmp.str(), ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT);
|
||||
}
|
||||
sequence_id++;
|
||||
|
||||
if (payload_length == 0)
|
||||
return false;
|
||||
}
|
||||
else if (offset == payload_length)
|
||||
{
|
||||
@ -208,6 +214,7 @@ class ClientPacket
|
||||
{
|
||||
public:
|
||||
ClientPacket() = default;
|
||||
|
||||
ClientPacket(ClientPacket &&) = default;
|
||||
|
||||
virtual void read(ReadBuffer & in, uint8_t & sequence_id)
|
||||
@ -257,6 +264,7 @@ public:
|
||||
{
|
||||
return total_left;
|
||||
}
|
||||
|
||||
private:
|
||||
WriteBuffer & out;
|
||||
uint8_t & sequence_id;
|
||||
@ -452,9 +460,6 @@ protected:
|
||||
buffer.write(static_cast<char>(auth_plugin_data.size()));
|
||||
writeChar(0x0, 10, buffer);
|
||||
writeString(auth_plugin_data.substr(AUTH_PLUGIN_DATA_PART_1_LENGTH, auth_plugin_data.size() - AUTH_PLUGIN_DATA_PART_1_LENGTH), buffer);
|
||||
// A workaround for PHP mysqlnd extension bug which occurs when sha256_password is used as a default authentication plugin.
|
||||
// Instead of using client response for mysql_native_password plugin, the server will always generate authentication method mismatch
|
||||
// and switch to sha256_password to simulate that mysql_native_password is used as a default plugin.
|
||||
writeString(auth_plugin_name, buffer);
|
||||
writeChar(0x0, 1, buffer);
|
||||
}
|
||||
@ -843,5 +848,230 @@ protected:
|
||||
}
|
||||
};
|
||||
|
||||
namespace Authentication
|
||||
{
|
||||
|
||||
class IPlugin
|
||||
{
|
||||
public:
|
||||
virtual String getName() = 0;
|
||||
|
||||
virtual String getAuthPluginData() = 0;
|
||||
|
||||
virtual void authenticate(const String & user_name, std::optional<String> auth_response, Context & context, std::shared_ptr<PacketSender> packet_sender, bool is_secure_connection,
|
||||
const Poco::Net::SocketAddress & address) = 0;
|
||||
|
||||
virtual ~IPlugin() = default;
|
||||
};
|
||||
|
||||
/// https://dev.mysql.com/doc/internals/en/secure-password-authentication.html
|
||||
class Native41 : public IPlugin
|
||||
{
|
||||
public:
|
||||
Native41()
|
||||
{
|
||||
scramble.resize(SCRAMBLE_LENGTH + 1, 0);
|
||||
Poco::RandomInputStream generator;
|
||||
|
||||
for (size_t i = 0; i < SCRAMBLE_LENGTH; i++)
|
||||
generator >> scramble[i];
|
||||
}
|
||||
|
||||
String getName() override
|
||||
{
|
||||
return "mysql_native_password";
|
||||
}
|
||||
|
||||
String getAuthPluginData() override
|
||||
{
|
||||
return scramble;
|
||||
}
|
||||
|
||||
void authenticate(
|
||||
const String & user_name,
|
||||
std::optional<String> auth_response,
|
||||
Context & context,
|
||||
std::shared_ptr<PacketSender> packet_sender,
|
||||
bool /* is_secure_connection */,
|
||||
const Poco::Net::SocketAddress & address) override
|
||||
{
|
||||
if (!auth_response)
|
||||
{
|
||||
packet_sender->sendPacket(AuthSwitchRequest(getName(), scramble), true);
|
||||
AuthSwitchResponse response;
|
||||
packet_sender->receivePacket(response);
|
||||
auth_response = response.value;
|
||||
}
|
||||
|
||||
if (auth_response->empty())
|
||||
{
|
||||
context.setUser(user_name, "", address, "");
|
||||
return;
|
||||
}
|
||||
|
||||
if (auth_response->size() != Poco::SHA1Engine::DIGEST_SIZE)
|
||||
throw Exception("Wrong size of auth response. Expected: " + std::to_string(Poco::SHA1Engine::DIGEST_SIZE) + " bytes, received: " + std::to_string(auth_response->size()) + " bytes.",
|
||||
ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
|
||||
auto user = context.getUser(user_name);
|
||||
|
||||
if (user->password_double_sha1_hex.empty())
|
||||
throw Exception("Cannot use " + getName() + " auth plugin for user " + user_name + " since its password isn't specified using double SHA1.", ErrorCodes::UNKNOWN_EXCEPTION);
|
||||
|
||||
Poco::SHA1Engine::Digest double_sha1_value = Poco::DigestEngine::digestFromHex(user->password_double_sha1_hex);
|
||||
assert(double_sha1_value.size() == Poco::SHA1Engine::DIGEST_SIZE);
|
||||
|
||||
Poco::SHA1Engine engine;
|
||||
engine.update(scramble.data(), SCRAMBLE_LENGTH);
|
||||
engine.update(double_sha1_value.data(), double_sha1_value.size());
|
||||
|
||||
String password_sha1(Poco::SHA1Engine::DIGEST_SIZE, 0x0);
|
||||
const Poco::SHA1Engine::Digest & digest = engine.digest();
|
||||
for (size_t i = 0; i < password_sha1.size(); i++)
|
||||
{
|
||||
password_sha1[i] = digest[i] ^ static_cast<unsigned char>((*auth_response)[i]);
|
||||
}
|
||||
context.setUser(user_name, password_sha1, address, "");
|
||||
}
|
||||
private:
|
||||
String scramble;
|
||||
};
|
||||
|
||||
/// Caching SHA2 plugin is not used because it would be possible to authenticate knowing hash from users.xml.
|
||||
/// https://dev.mysql.com/doc/internals/en/sha256.html
|
||||
class Sha256Password : public IPlugin
|
||||
{
|
||||
public:
|
||||
Sha256Password(RSA & public_key_, RSA & private_key_, Logger * log_)
|
||||
: public_key(public_key_)
|
||||
, private_key(private_key_)
|
||||
, log(log_)
|
||||
{
|
||||
/** Native authentication sent 20 bytes + '\0' character = 21 bytes.
|
||||
* This plugin must do the same to stay consistent with historical behavior if it is set to operate as a default plugin. [1]
|
||||
* https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L3994
|
||||
*/
|
||||
scramble.resize(SCRAMBLE_LENGTH + 1, 0);
|
||||
Poco::RandomInputStream generator;
|
||||
|
||||
for (size_t i = 0; i < SCRAMBLE_LENGTH; i++)
|
||||
generator >> scramble[i];
|
||||
}
|
||||
|
||||
String getName() override
|
||||
{
|
||||
return "sha256_password";
|
||||
}
|
||||
|
||||
String getAuthPluginData() override
|
||||
{
|
||||
return scramble;
|
||||
}
|
||||
|
||||
void authenticate(
|
||||
const String & user_name,
|
||||
std::optional<String> auth_response,
|
||||
Context & context,
|
||||
std::shared_ptr<PacketSender> packet_sender,
|
||||
bool is_secure_connection,
|
||||
const Poco::Net::SocketAddress & address) override
|
||||
{
|
||||
if (!auth_response)
|
||||
{
|
||||
packet_sender->sendPacket(AuthSwitchRequest(getName(), scramble), true);
|
||||
|
||||
if (packet_sender->in->eof())
|
||||
throw Exception("Client doesn't support authentication method " + getName() + " used by ClickHouse. Specifying user password using 'password_double_sha1_hex' may fix the problem.",
|
||||
ErrorCodes::MYSQL_CLIENT_INSUFFICIENT_CAPABILITIES);
|
||||
|
||||
AuthSwitchResponse response;
|
||||
packet_sender->receivePacket(response);
|
||||
auth_response = response.value;
|
||||
LOG_TRACE(log, "Authentication method mismatch.");
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(log, "Authentication method match.");
|
||||
}
|
||||
|
||||
if (auth_response == "\1")
|
||||
{
|
||||
LOG_TRACE(log, "Client requests public key.");
|
||||
|
||||
BIO * mem = BIO_new(BIO_s_mem());
|
||||
SCOPE_EXIT(BIO_free(mem));
|
||||
if (PEM_write_bio_RSA_PUBKEY(mem, &public_key) != 1)
|
||||
{
|
||||
throw Exception("Failed to write public key to memory. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
|
||||
}
|
||||
char * pem_buf = nullptr;
|
||||
long pem_size = BIO_get_mem_data(mem, &pem_buf);
|
||||
String pem(pem_buf, pem_size);
|
||||
|
||||
LOG_TRACE(log, "Key: " << pem);
|
||||
|
||||
AuthMoreData data(pem);
|
||||
packet_sender->sendPacket(data, true);
|
||||
|
||||
AuthSwitchResponse response;
|
||||
packet_sender->receivePacket(response);
|
||||
auth_response = response.value;
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(log, "Client didn't request public key.");
|
||||
}
|
||||
|
||||
String password;
|
||||
|
||||
/** Decrypt password, if it's not empty.
|
||||
* The original intention was that the password is a string[NUL] but this never got enforced properly so now we have to accept that
|
||||
* an empty packet is a blank password, thus the check for auth_response.empty() has to be made too.
|
||||
* https://github.com/mysql/mysql-server/blob/8.0/sql/auth/sql_authentication.cc#L4017
|
||||
*/
|
||||
if (!is_secure_connection && !auth_response->empty() && auth_response != String("\0", 1))
|
||||
{
|
||||
LOG_TRACE(log, "Received nonempty password");
|
||||
auto ciphertext = reinterpret_cast<unsigned char *>(auth_response->data());
|
||||
|
||||
unsigned char plaintext[RSA_size(&private_key)];
|
||||
int plaintext_size = RSA_private_decrypt(auth_response->size(), ciphertext, plaintext, &private_key, RSA_PKCS1_OAEP_PADDING);
|
||||
if (plaintext_size == -1)
|
||||
{
|
||||
throw Exception("Failed to decrypt auth data. Error: " + getOpenSSLErrors(), ErrorCodes::OPENSSL_ERROR);
|
||||
}
|
||||
|
||||
password.resize(plaintext_size);
|
||||
for (int i = 0; i < plaintext_size; i++)
|
||||
{
|
||||
password[i] = plaintext[i] ^ static_cast<unsigned char>(scramble[i % scramble.size()]);
|
||||
}
|
||||
}
|
||||
else if (is_secure_connection)
|
||||
{
|
||||
password = *auth_response;
|
||||
}
|
||||
else
|
||||
{
|
||||
LOG_TRACE(log, "Received empty password");
|
||||
}
|
||||
|
||||
if (!password.empty() && password.back() == 0)
|
||||
{
|
||||
password.pop_back();
|
||||
}
|
||||
|
||||
context.setUser(user_name, password, address, "");
|
||||
}
|
||||
|
||||
private:
|
||||
RSA & public_key;
|
||||
RSA & private_key;
|
||||
Logger * log;
|
||||
String scramble;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -555,7 +555,7 @@ public:
|
||||
for (const auto & member : members())
|
||||
{
|
||||
if (member.isChanged(castToDerived()))
|
||||
found_changes.emplace_back(member.name.toString(), member.get_field(castToDerived()));
|
||||
found_changes.push_back({member.name.toString(), member.get_field(castToDerived())});
|
||||
}
|
||||
return found_changes;
|
||||
}
|
||||
|
@ -73,6 +73,15 @@ public:
|
||||
|
||||
Block getHeader() const override { return children.at(0)->getHeader(); }
|
||||
|
||||
void cancel(bool kill) override
|
||||
{
|
||||
IBlockInputStream::cancel(kill);
|
||||
|
||||
/// Wait for some backgroud calculations to be sure,
|
||||
/// that after end of stream nothing is being executing.
|
||||
if (started)
|
||||
pool.wait();
|
||||
}
|
||||
|
||||
~AsynchronousBlockInputStream() override
|
||||
{
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Poco/Logger.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h> /// SubqueriesForSets
|
||||
#include <Interpreters/SubqueryForSet.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
@ -102,8 +102,6 @@ Block FinishSortingBlockInputStream::readImpl()
|
||||
if (block.rows() == 0)
|
||||
continue;
|
||||
|
||||
// We need to sort each block separately before merging.
|
||||
sortBlock(block, description_to_sort, limit);
|
||||
|
||||
removeConstantsFromBlock(block);
|
||||
|
||||
|
@ -110,8 +110,7 @@ ParallelAggregatingBlockInputStream::TemporaryFileStream::TemporaryFileStream(co
|
||||
void ParallelAggregatingBlockInputStream::Handler::onBlock(Block & block, size_t thread_num)
|
||||
{
|
||||
parent.aggregator.executeOnBlock(block, *parent.many_data[thread_num],
|
||||
parent.threads_data[thread_num].key_columns, parent.threads_data[thread_num].aggregate_columns,
|
||||
parent.threads_data[thread_num].key, parent.no_more_keys);
|
||||
parent.threads_data[thread_num].key_columns, parent.threads_data[thread_num].aggregate_columns, parent.no_more_keys);
|
||||
|
||||
parent.threads_data[thread_num].src_rows += block.rows();
|
||||
parent.threads_data[thread_num].src_bytes += block.bytes();
|
||||
@ -205,8 +204,7 @@ void ParallelAggregatingBlockInputStream::execute()
|
||||
/// To do this, we pass a block with zero rows to aggregate.
|
||||
if (total_src_rows == 0 && params.keys_size == 0 && !params.empty_result_for_aggregation_by_empty_set)
|
||||
aggregator.executeOnBlock(children.at(0)->getHeader(), *many_data[0],
|
||||
threads_data[0].key_columns, threads_data[0].aggregate_columns,
|
||||
threads_data[0].key, no_more_keys);
|
||||
threads_data[0].key_columns, threads_data[0].aggregate_columns, no_more_keys);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -80,13 +80,11 @@ private:
|
||||
size_t src_rows = 0;
|
||||
size_t src_bytes = 0;
|
||||
|
||||
StringRefs key;
|
||||
ColumnRawPtrs key_columns;
|
||||
Aggregator::AggregateColumns aggregate_columns;
|
||||
|
||||
ThreadData(size_t keys_size_, size_t aggregates_size_)
|
||||
{
|
||||
key.resize(keys_size_);
|
||||
key_columns.resize(keys_size_);
|
||||
aggregate_columns.resize(aggregates_size_);
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ try
|
||||
|
||||
NamesAndTypesList source_columns = {{"number", std::make_shared<DataTypeUInt64>()}};
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns);
|
||||
ExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
ExpressionActionsChain chain(context);
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain);
|
||||
|
@ -44,7 +44,7 @@ try
|
||||
|
||||
NamesAndTypesList source_columns = {{"number", std::make_shared<DataTypeUInt64>()}};
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, source_columns);
|
||||
ExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
ExpressionActionsChain chain(context);
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain);
|
||||
|
@ -63,9 +63,9 @@ void ComplexKeyCacheDictionary::setAttributeValue(Attribute & attribute, const s
|
||||
const auto str_size = string.size();
|
||||
if (str_size != 0)
|
||||
{
|
||||
auto string_ptr = string_arena->alloc(str_size + 1);
|
||||
std::copy(string.data(), string.data() + str_size + 1, string_ptr);
|
||||
string_ref = StringRef{string_ptr, str_size};
|
||||
auto str_ptr = string_arena->alloc(str_size);
|
||||
std::copy(string.data(), string.data() + str_size, str_ptr);
|
||||
string_ref = StringRef{str_ptr, str_size};
|
||||
}
|
||||
else
|
||||
string_ref = {};
|
||||
|
@ -1,5 +1,6 @@
|
||||
#include "LibraryDictionarySource.h"
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Core/Defines.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/File.h>
|
||||
#include <common/logger_useful.h>
|
||||
@ -134,7 +135,7 @@ LibraryDictionarySource::LibraryDictionarySource(
|
||||
ErrorCodes::FILE_DOESNT_EXIST);
|
||||
description.init(sample_block);
|
||||
library = std::make_shared<SharedLibrary>(path, RTLD_LAZY
|
||||
#if defined(RTLD_DEEPBIND) // Does not exists in freebsd
|
||||
#if defined(RTLD_DEEPBIND) && !defined(ADDRESS_SANITIZER) // Does not exists in FreeBSD. Cannot work with Address Sanitizer.
|
||||
| RTLD_DEEPBIND
|
||||
#endif
|
||||
);
|
||||
|
@ -83,7 +83,17 @@ BlockInputStreamPtr FormatFactory::getInput(
|
||||
return std::make_shared<NativeBlockInputStream>(buf, sample, 0);
|
||||
|
||||
if (!getCreators(name).input_processor_creator)
|
||||
return getInput(name, buf, sample, context, max_block_size, rows_portion_size, std::move(callback));
|
||||
{
|
||||
const auto & input_getter = getCreators(name).inout_creator;
|
||||
if (!input_getter)
|
||||
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
FormatSettings format_settings = getInputFormatSetting(settings);
|
||||
|
||||
return input_getter(
|
||||
buf, sample, context, max_block_size, rows_portion_size, callback ? callback : ReadCallback(), format_settings);
|
||||
}
|
||||
|
||||
auto format = getInputFormat(name, buf, sample, context, max_block_size, rows_portion_size, std::move(callback));
|
||||
return std::make_shared<InputStreamFromInputFormat>(std::move(format));
|
||||
@ -106,13 +116,22 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer &
|
||||
}
|
||||
|
||||
if (!getCreators(name).output_processor_creator)
|
||||
return getOutput(name, buf, sample, context);
|
||||
{
|
||||
const auto & output_getter = getCreators(name).output_creator;
|
||||
if (!output_getter)
|
||||
throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT);
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
FormatSettings format_settings = getOutputFormatSetting(settings);
|
||||
|
||||
/** Materialization is needed, because formats can use the functions `IDataType`,
|
||||
* which only work with full columns.
|
||||
*/
|
||||
return std::make_shared<MaterializingBlockOutputStream>(
|
||||
output_getter(buf, sample, context, format_settings), sample);
|
||||
}
|
||||
|
||||
auto format = getOutputFormat(name, buf, sample, context);
|
||||
|
||||
/** Materialization is needed, because formats can use the functions `IDataType`,
|
||||
* which only work with full columns.
|
||||
*/
|
||||
return std::make_shared<MaterializingBlockOutputStream>(std::make_shared<OutputStreamToOutputFormat>(format), sample);
|
||||
}
|
||||
|
||||
|
@ -25,6 +25,7 @@ target_link_libraries(clickhouse_functions
|
||||
PRIVATE
|
||||
${ZLIB_LIBRARIES}
|
||||
${Boost_FILESYSTEM_LIBRARY}
|
||||
${CMAKE_DL_LIBS}
|
||||
)
|
||||
|
||||
if (OPENSSL_CRYPTO_LIBRARY)
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include "config_functions.h"
|
||||
#include <Common/CpuId.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
@ -15,7 +16,6 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
@ -64,7 +64,7 @@ public:
|
||||
{
|
||||
/// Choose JSONParser.
|
||||
#if USE_SIMDJSON
|
||||
if (context.getSettings().allow_simdjson && Cpu::CpuFlagsCache::have_AVX2)
|
||||
if (context.getSettings().allow_simdjson && Cpu::CpuFlagsCache::have_SSE42 && Cpu::CpuFlagsCache::have_PCLMUL)
|
||||
{
|
||||
Executor<SimdJSONParser>::run(block, arguments, result_pos, input_rows_count);
|
||||
return;
|
||||
|
@ -307,10 +307,8 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
||||
ColumnUInt32::Container & res_values)
|
||||
{
|
||||
/// Offsets at the depth we want to look.
|
||||
const size_t current_offset_depth = arrays_depths.max_array_depth;
|
||||
const auto & offsets = *offsets_by_depth[current_offset_depth - 1];
|
||||
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
const size_t depth_to_look = arrays_depths.max_array_depth;
|
||||
const auto & offsets = *offsets_by_depth[depth_to_look - 1];
|
||||
|
||||
using Map = ClearableHashMap<
|
||||
UInt128,
|
||||
@ -320,14 +318,17 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
||||
HashTableAllocatorWithStackMemory<(1ULL << INITIAL_SIZE_DEGREE) * sizeof(UInt128)>>;
|
||||
Map indices;
|
||||
|
||||
std::vector<size_t> indices_by_depth(arrays_depths.max_array_depth);
|
||||
std::vector<size_t> current_offset_n_by_depth(arrays_depths.max_array_depth);
|
||||
std::vector<size_t> last_offset_by_depth(arrays_depths.max_array_depth, 0); // For skipping empty arrays
|
||||
std::vector<size_t> indices_by_depth(depth_to_look);
|
||||
std::vector<size_t> current_offset_n_by_depth(depth_to_look);
|
||||
std::vector<size_t> last_offset_by_depth(depth_to_look, 0); // For skipping empty arrays
|
||||
|
||||
/// For arrayEnumerateDense variant: to calculate every distinct value.
|
||||
UInt32 rank = 0;
|
||||
|
||||
std::vector<size_t> columns_indices(columns.size());
|
||||
|
||||
/// For each array at the depth we want to look.
|
||||
ColumnArray::Offset prev_off = 0;
|
||||
for (size_t off : offsets)
|
||||
{
|
||||
bool want_clear = false;
|
||||
@ -336,17 +337,29 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
||||
if (prev_off == off)
|
||||
{
|
||||
want_clear = true;
|
||||
if (arrays_depths.max_array_depth > 1)
|
||||
++indices_by_depth[0];
|
||||
|
||||
for (ssize_t depth = current_offset_depth - 1; depth >= 0; --depth)
|
||||
if (depth_to_look >= 2)
|
||||
{
|
||||
const auto offsets_by_depth_size = offsets_by_depth[depth]->size();
|
||||
while (last_offset_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]])
|
||||
/// Advance to the next element of the parent array.
|
||||
for (ssize_t depth = depth_to_look - 2; depth >= 0; --depth)
|
||||
{
|
||||
if (current_offset_n_by_depth[depth] + 1 >= offsets_by_depth_size)
|
||||
break; // only one empty array: SELECT arrayEnumerateUniqRanked([]);
|
||||
++current_offset_n_by_depth[depth];
|
||||
/// Skipping offsets for empty arrays
|
||||
while (last_offset_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]])
|
||||
{
|
||||
++current_offset_n_by_depth[depth];
|
||||
}
|
||||
|
||||
++indices_by_depth[depth];
|
||||
|
||||
if (indices_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]])
|
||||
{
|
||||
last_offset_by_depth[depth] = (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]];
|
||||
++current_offset_n_by_depth[depth];
|
||||
}
|
||||
else
|
||||
{
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -377,7 +390,7 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
|
||||
|
||||
// Debug: DUMP(off, prev_off, j, columns_indices, res_values[j], columns);
|
||||
|
||||
for (ssize_t depth = current_offset_depth - 1; depth >= 0; --depth)
|
||||
for (ssize_t depth = depth_to_look - 1; depth >= 0; --depth)
|
||||
{
|
||||
/// Skipping offsets for empty arrays
|
||||
while (last_offset_by_depth[depth] == (*offsets_by_depth[depth])[current_offset_n_by_depth[depth]])
|
||||
|
@ -4,6 +4,10 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_CAST;
|
||||
}
|
||||
|
||||
/// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h).
|
||||
/// This function provides "AND" operation for BoolMasks.
|
||||
@ -17,6 +21,8 @@ namespace DB
|
||||
template <typename Result = ResultType>
|
||||
static inline Result apply(A left, B right)
|
||||
{
|
||||
if constexpr (!std::is_same_v<A, ResultType> || !std::is_same_v<B, ResultType>)
|
||||
throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskAnd.", ErrorCodes::BAD_CAST);
|
||||
return static_cast<ResultType>(
|
||||
((static_cast<ResultType>(left) & static_cast<ResultType>(right)) & 1)
|
||||
| ((((static_cast<ResultType>(left) >> 1) | (static_cast<ResultType>(right) >> 1)) & 1) << 1));
|
||||
|
@ -4,6 +4,10 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_CAST;
|
||||
}
|
||||
|
||||
/// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h).
|
||||
/// This function provides "OR" operation for BoolMasks.
|
||||
@ -17,6 +21,8 @@ namespace DB
|
||||
template <typename Result = ResultType>
|
||||
static inline Result apply(A left, B right)
|
||||
{
|
||||
if constexpr (!std::is_same_v<A, ResultType> || !std::is_same_v<B, ResultType>)
|
||||
throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitBoolMaskOr.", ErrorCodes::BAD_CAST);
|
||||
return static_cast<ResultType>(
|
||||
((static_cast<ResultType>(left) | static_cast<ResultType>(right)) & 1)
|
||||
| ((((static_cast<ResultType>(left) >> 1) & (static_cast<ResultType>(right) >> 1)) & 1) << 1));
|
||||
|
@ -4,6 +4,10 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_CAST;
|
||||
}
|
||||
|
||||
/// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h).
|
||||
/// This function provides "NOT" operation for BoolMasks by swapping last two bits ("can be true" <-> "can be false").
|
||||
@ -14,6 +18,8 @@ namespace DB
|
||||
|
||||
static inline ResultType NO_SANITIZE_UNDEFINED apply(A a)
|
||||
{
|
||||
if constexpr (!std::is_same_v<A, ResultType>)
|
||||
throw DB::Exception("It's a bug! Only UInt8 type is supported by __bitSwapLastTwo.", ErrorCodes::BAD_CAST);
|
||||
return static_cast<ResultType>(
|
||||
((static_cast<ResultType>(a) & 1) << 1) | ((static_cast<ResultType>(a) >> 1) & 1));
|
||||
}
|
||||
|
@ -4,6 +4,10 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_CAST;
|
||||
}
|
||||
|
||||
/// Working with UInt8: last bit = can be true, previous = can be false (Like dbms/src/Storages/MergeTree/BoolMask.h).
|
||||
/// This function wraps bool atomic functions
|
||||
@ -15,7 +19,9 @@ namespace DB
|
||||
|
||||
static inline ResultType NO_SANITIZE_UNDEFINED apply(A a)
|
||||
{
|
||||
return a == static_cast<UInt8>(0) ? static_cast<ResultType>(0b10) : static_cast<ResultType >(0b1);
|
||||
if constexpr (!std::is_integral_v<A>)
|
||||
throw DB::Exception("It's a bug! Only integer types are supported by __bitWrapperFunc.", ErrorCodes::BAD_CAST);
|
||||
return a == 0 ? static_cast<ResultType>(0b10) : static_cast<ResultType >(0b1);
|
||||
}
|
||||
|
||||
#if USE_EMBEDDED_COMPILER
|
||||
|
54
dbms/src/Functions/currentUser.cpp
Normal file
54
dbms/src/Functions/currentUser.cpp
Normal file
@ -0,0 +1,54 @@
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class FunctionCurrentUser : public IFunction
|
||||
{
|
||||
const String user_name;
|
||||
|
||||
public:
|
||||
static constexpr auto name = "currentUser";
|
||||
static FunctionPtr create(const Context & context)
|
||||
{
|
||||
return std::make_shared<FunctionCurrentUser>(context.getClientInfo().initial_user);
|
||||
}
|
||||
|
||||
explicit FunctionCurrentUser(const String & user_name_) : user_name{user_name_}
|
||||
{
|
||||
}
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
return name;
|
||||
}
|
||||
size_t getNumberOfArguments() const override
|
||||
{
|
||||
return 0;
|
||||
}
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override
|
||||
{
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
bool isDeterministic() const override { return false; }
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, user_name);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
void registerFunctionCurrentUser(FunctionFactory & factory)
|
||||
{
|
||||
factory.registerFunction<FunctionCurrentUser>();
|
||||
factory.registerAlias("user", FunctionCurrentUser::name, FunctionFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
@ -43,7 +43,7 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
/// nullIf(col1, col2) == if(col1 == col2, NULL, col1)
|
||||
/// nullIf(col1, col2) == if(col1 != col2, col1, NULL)
|
||||
|
||||
Block temp_block = block;
|
||||
|
||||
@ -51,7 +51,7 @@ public:
|
||||
temp_block.insert({nullptr, std::make_shared<DataTypeUInt8>(), ""});
|
||||
|
||||
{
|
||||
auto equals_func = FunctionFactory::instance().get("equals", context)->build(
|
||||
auto equals_func = FunctionFactory::instance().get("notEquals", context)->build(
|
||||
{temp_block.getByPosition(arguments[0]), temp_block.getByPosition(arguments[1])});
|
||||
equals_func->execute(temp_block, {arguments[0], arguments[1]}, res_pos, input_rows_count);
|
||||
}
|
||||
@ -69,7 +69,7 @@ public:
|
||||
|
||||
auto func_if = FunctionFactory::instance().get("if", context)->build(
|
||||
{temp_block.getByPosition(res_pos), temp_block.getByPosition(null_pos), temp_block.getByPosition(arguments[0])});
|
||||
func_if->execute(temp_block, {res_pos, null_pos, arguments[0]}, result, input_rows_count);
|
||||
func_if->execute(temp_block, {res_pos, arguments[0], null_pos}, result, input_rows_count);
|
||||
|
||||
block.getByPosition(result).column = std::move(temp_block.getByPosition(result).column);
|
||||
}
|
||||
|
@ -6,6 +6,7 @@ namespace DB
|
||||
class FunctionFactory;
|
||||
|
||||
void registerFunctionCurrentDatabase(FunctionFactory &);
|
||||
void registerFunctionCurrentUser(FunctionFactory &);
|
||||
void registerFunctionHostName(FunctionFactory &);
|
||||
void registerFunctionVisibleWidth(FunctionFactory &);
|
||||
void registerFunctionToTypeName(FunctionFactory &);
|
||||
@ -56,6 +57,7 @@ void registerFunctionConvertCharset(FunctionFactory &);
|
||||
void registerFunctionsMiscellaneous(FunctionFactory & factory)
|
||||
{
|
||||
registerFunctionCurrentDatabase(factory);
|
||||
registerFunctionCurrentUser(factory);
|
||||
registerFunctionHostName(factory);
|
||||
registerFunctionVisibleWidth(factory);
|
||||
registerFunctionToTypeName(factory);
|
||||
|
@ -0,0 +1,29 @@
|
||||
#include <IO/WriteBufferFromFileDescriptorDiscardOnFailure.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
extern const Event CannotWriteToWriteBufferDiscard;
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void WriteBufferFromFileDescriptorDiscardOnFailure::nextImpl()
|
||||
{
|
||||
size_t bytes_written = 0;
|
||||
while (bytes_written != offset())
|
||||
{
|
||||
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
|
||||
|
||||
if ((-1 == res || 0 == res) && errno != EINTR)
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::CannotWriteToWriteBufferDiscard);
|
||||
break; /// Discard
|
||||
}
|
||||
|
||||
if (res > 0)
|
||||
bytes_written += res;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
23
dbms/src/IO/WriteBufferFromFileDescriptorDiscardOnFailure.h
Normal file
23
dbms/src/IO/WriteBufferFromFileDescriptorDiscardOnFailure.h
Normal file
@ -0,0 +1,23 @@
|
||||
#pragma once
|
||||
|
||||
#include <IO/WriteBufferFromFileDescriptor.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Write to file descriptor but drop the data if write would block or fail.
|
||||
* To use within signal handler. Motivating example: a signal handler invoked during execution of malloc
|
||||
* should not block because some mutex (or even worse - a spinlock) may be held.
|
||||
*/
|
||||
class WriteBufferFromFileDescriptorDiscardOnFailure : public WriteBufferFromFileDescriptor
|
||||
{
|
||||
protected:
|
||||
void nextImpl() override;
|
||||
|
||||
public:
|
||||
using WriteBufferFromFileDescriptor::WriteBufferFromFileDescriptor;
|
||||
~WriteBufferFromFileDescriptorDiscardOnFailure() override {}
|
||||
};
|
||||
|
||||
}
|
@ -59,10 +59,10 @@ target_link_libraries (write_int PRIVATE clickhouse_common_io)
|
||||
|
||||
if (OS_LINUX OR OS_FREEBSD)
|
||||
add_executable(write_buffer_aio write_buffer_aio.cpp)
|
||||
target_link_libraries (write_buffer_aio PRIVATE clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY})
|
||||
target_link_libraries (write_buffer_aio PRIVATE clickhouse_common_io stdc++fs)
|
||||
|
||||
add_executable(read_buffer_aio read_buffer_aio.cpp)
|
||||
target_link_libraries (read_buffer_aio PRIVATE clickhouse_common_io ${Boost_FILESYSTEM_LIBRARY})
|
||||
target_link_libraries (read_buffer_aio PRIVATE clickhouse_common_io stdc++fs)
|
||||
endif ()
|
||||
|
||||
add_executable (zlib_buffers zlib_buffers.cpp)
|
||||
|
@ -1,3 +1,5 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsMiscellaneous.h>
|
||||
|
||||
@ -19,8 +21,6 @@
|
||||
#include <Storages/StorageSet.h>
|
||||
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/DumpASTNode.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
@ -228,346 +228,350 @@ const Block & ScopeStack::getSampleBlock() const
|
||||
return stack.back().actions->getSampleBlock();
|
||||
}
|
||||
|
||||
|
||||
ActionsVisitor::ActionsVisitor(
|
||||
const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
|
||||
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_)
|
||||
: context(context_),
|
||||
set_size_limit(set_size_limit_),
|
||||
subquery_depth(subquery_depth_),
|
||||
source_columns(source_columns_),
|
||||
prepared_sets(prepared_sets_),
|
||||
subqueries_for_sets(subqueries_for_sets_),
|
||||
no_subqueries(no_subqueries_),
|
||||
only_consts(only_consts_),
|
||||
no_storage_or_local(no_storage_or_local_),
|
||||
visit_depth(0),
|
||||
ostr(ostr_),
|
||||
actions_stack(actions, context)
|
||||
struct CachedColumnName
|
||||
{
|
||||
String cached;
|
||||
|
||||
const String & get(const ASTPtr & ast)
|
||||
{
|
||||
if (cached.empty())
|
||||
cached = ast->getColumnName();
|
||||
return cached;
|
||||
}
|
||||
};
|
||||
|
||||
bool ActionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr & child)
|
||||
{
|
||||
/// Visit children themself
|
||||
if (node->as<ASTIdentifier>() ||
|
||||
node->as<ASTFunction>() ||
|
||||
node->as<ASTLiteral>())
|
||||
return false;
|
||||
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
if (child->as<ASTTableExpression>() ||
|
||||
child->as<ASTSelectQuery>())
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
void ActionsVisitor::visit(const ASTPtr & ast)
|
||||
void ActionsMatcher::visit(const ASTPtr & ast, Data & data)
|
||||
{
|
||||
DumpASTNode dump(*ast, ostr, visit_depth, "getActions");
|
||||
if (const auto * identifier = ast->as<ASTIdentifier>())
|
||||
visit(*identifier, ast, data);
|
||||
else if (const auto * node = ast->as<ASTFunction>())
|
||||
visit(*node, ast, data);
|
||||
else if (const auto * literal = ast->as<ASTLiteral>())
|
||||
visit(*literal, ast, data);
|
||||
}
|
||||
|
||||
String ast_column_name;
|
||||
auto getColumnName = [&ast, &ast_column_name]()
|
||||
{
|
||||
if (ast_column_name.empty())
|
||||
ast_column_name = ast->getColumnName();
|
||||
|
||||
return ast_column_name;
|
||||
};
|
||||
|
||||
/// If the result of the calculation already exists in the block.
|
||||
if ((ast->as<ASTFunction>() || ast->as<ASTLiteral>()) && actions_stack.getSampleBlock().has(getColumnName()))
|
||||
void ActionsMatcher::visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
if (const auto * identifier = ast->as<ASTIdentifier>())
|
||||
if (!data.only_consts)
|
||||
{
|
||||
if (!only_consts && !actions_stack.getSampleBlock().has(getColumnName()))
|
||||
{
|
||||
/// The requested column is not in the block.
|
||||
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
|
||||
/// The requested column is not in the block.
|
||||
/// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY.
|
||||
|
||||
bool found = false;
|
||||
for (const auto & column_name_type : source_columns)
|
||||
if (column_name_type.name == getColumnName())
|
||||
found = true;
|
||||
bool found = false;
|
||||
for (const auto & column_name_type : data.source_columns)
|
||||
if (column_name_type.name == column_name.get(ast))
|
||||
found = true;
|
||||
|
||||
if (found)
|
||||
throw Exception("Column " + getColumnName() + " is not under aggregate function and not in GROUP BY.",
|
||||
ErrorCodes::NOT_AN_AGGREGATE);
|
||||
if (found)
|
||||
throw Exception("Column " + column_name.get(ast) + " is not under aggregate function and not in GROUP BY.",
|
||||
ErrorCodes::NOT_AN_AGGREGATE);
|
||||
|
||||
/// Special check for WITH statement alias. Add alias action to be able to use this alias.
|
||||
if (identifier->prefer_alias_to_column_name && !identifier->alias.empty())
|
||||
actions_stack.addAction(ExpressionAction::addAliases({{identifier->name, identifier->alias}}));
|
||||
}
|
||||
/// Special check for WITH statement alias. Add alias action to be able to use this alias.
|
||||
if (identifier.prefer_alias_to_column_name && !identifier.alias.empty())
|
||||
data.addAction(ExpressionAction::addAliases({{identifier.name, identifier.alias}}));
|
||||
}
|
||||
else if (const auto * node = ast->as<ASTFunction>())
|
||||
}
|
||||
|
||||
void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
if (node.name == "lambda")
|
||||
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
|
||||
/// Function arrayJoin.
|
||||
if (node.name == "arrayJoin")
|
||||
{
|
||||
if (node->name == "lambda")
|
||||
throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION);
|
||||
if (node.arguments->children.size() != 1)
|
||||
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
/// Function arrayJoin.
|
||||
if (node->name == "arrayJoin")
|
||||
ASTPtr arg = node.arguments->children.at(0);
|
||||
visit(arg, data);
|
||||
if (!data.only_consts)
|
||||
{
|
||||
if (node->arguments->children.size() != 1)
|
||||
throw Exception("arrayJoin requires exactly 1 argument", ErrorCodes::TYPE_MISMATCH);
|
||||
String result_name = column_name.get(ast);
|
||||
data.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name));
|
||||
NameSet joined_columns;
|
||||
joined_columns.insert(result_name);
|
||||
data.addAction(ExpressionAction::arrayJoin(joined_columns, false, data.context));
|
||||
}
|
||||
|
||||
ASTPtr arg = node->arguments->children.at(0);
|
||||
visit(arg);
|
||||
if (!only_consts)
|
||||
return;
|
||||
}
|
||||
|
||||
SetPtr prepared_set;
|
||||
if (functionIsInOrGlobalInOperator(node.name))
|
||||
{
|
||||
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
||||
visit(node.arguments->children.at(0), data);
|
||||
|
||||
if (!data.no_subqueries)
|
||||
{
|
||||
/// Transform tuple or subquery into a set.
|
||||
prepared_set = makeSet(node, data);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!data.only_consts)
|
||||
{
|
||||
String result_name = getColumnName();
|
||||
actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name));
|
||||
NameSet joined_columns;
|
||||
joined_columns.insert(result_name);
|
||||
actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context));
|
||||
}
|
||||
/// We are in the part of the tree that we are not going to compute. You just need to define types.
|
||||
/// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function.
|
||||
|
||||
data.addAction(ExpressionAction::applyFunction(
|
||||
FunctionFactory::instance().get("ignoreExceptNull", data.context),
|
||||
{ node.arguments->children.at(0)->getColumnName() },
|
||||
column_name.get(ast)));
|
||||
}
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
SetPtr prepared_set;
|
||||
if (functionIsInOrGlobalInOperator(node->name))
|
||||
/// A special function `indexHint`. Everything that is inside it is not calculated
|
||||
/// (and is used only for index analysis, see KeyCondition).
|
||||
if (node.name == "indexHint")
|
||||
{
|
||||
data.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
|
||||
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(),
|
||||
column_name.get(ast))));
|
||||
return;
|
||||
}
|
||||
|
||||
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node.name))
|
||||
return;
|
||||
|
||||
/// Context object that we pass to function should live during query.
|
||||
const Context & function_context = data.context.hasQueryContext()
|
||||
? data.context.getQueryContext()
|
||||
: data.context;
|
||||
|
||||
FunctionBuilderPtr function_builder;
|
||||
try
|
||||
{
|
||||
function_builder = FunctionFactory::instance().get(node.name, function_context);
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
auto hints = AggregateFunctionFactory::instance().getHints(node.name);
|
||||
if (!hints.empty())
|
||||
e.addMessage("Or unknown aggregate function " + node.name + ". Maybe you meant: " + toString(hints));
|
||||
e.rethrow();
|
||||
}
|
||||
|
||||
Names argument_names;
|
||||
DataTypes argument_types;
|
||||
bool arguments_present = true;
|
||||
|
||||
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
|
||||
bool has_lambda_arguments = false;
|
||||
|
||||
for (size_t arg = 0; arg < node.arguments->children.size(); ++arg)
|
||||
{
|
||||
auto & child = node.arguments->children[arg];
|
||||
auto child_column_name = child->getColumnName();
|
||||
|
||||
const auto * lambda = child->as<ASTFunction>();
|
||||
if (lambda && lambda->name == "lambda")
|
||||
{
|
||||
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
|
||||
visit(node->arguments->children.at(0));
|
||||
/// If the argument is a lambda expression, just remember its approximate type.
|
||||
if (lambda->arguments->children.size() != 2)
|
||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!no_subqueries)
|
||||
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||
|
||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
has_lambda_arguments = true;
|
||||
argument_types.emplace_back(std::make_shared<DataTypeFunction>(DataTypes(lambda_args_tuple->arguments->children.size())));
|
||||
/// Select the name in the next cycle.
|
||||
argument_names.emplace_back();
|
||||
}
|
||||
else if (functionIsInOrGlobalInOperator(node.name) && arg == 1 && prepared_set)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeSet>();
|
||||
|
||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||
if (!prepared_set->empty())
|
||||
column.name = getUniqueName(data.getSampleBlock(), "__set");
|
||||
else
|
||||
column.name = child_column_name;
|
||||
|
||||
if (!data.hasColumn(column.name))
|
||||
{
|
||||
/// Transform tuple or subquery into a set.
|
||||
prepared_set = makeSet(node, actions_stack.getSampleBlock());
|
||||
column.column = ColumnSet::create(1, prepared_set);
|
||||
|
||||
data.addAction(ExpressionAction::addColumn(column));
|
||||
}
|
||||
|
||||
argument_types.push_back(column.type);
|
||||
argument_names.push_back(column.name);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If the argument is not a lambda expression, call it recursively and find out its type.
|
||||
visit(child, data);
|
||||
std::string name = child_column_name;
|
||||
if (data.hasColumn(name))
|
||||
{
|
||||
argument_types.push_back(data.getSampleBlock().getByName(name).type);
|
||||
argument_names.push_back(name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!only_consts)
|
||||
{
|
||||
/// We are in the part of the tree that we are not going to compute. You just need to define types.
|
||||
/// Do not subquery and create sets. We treat "IN" as "ignoreExceptNull" function.
|
||||
|
||||
actions_stack.addAction(ExpressionAction::applyFunction(
|
||||
FunctionFactory::instance().get("ignoreExceptNull", context),
|
||||
{ node->arguments->children.at(0)->getColumnName() },
|
||||
getColumnName()));
|
||||
}
|
||||
return;
|
||||
if (data.only_consts)
|
||||
arguments_present = false;
|
||||
else
|
||||
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/// A special function `indexHint`. Everything that is inside it is not calculated
|
||||
/// (and is used only for index analysis, see KeyCondition).
|
||||
if (node->name == "indexHint")
|
||||
if (data.only_consts && !arguments_present)
|
||||
return;
|
||||
|
||||
if (has_lambda_arguments && !data.only_consts)
|
||||
{
|
||||
function_builder->getLambdaArgumentTypes(argument_types);
|
||||
|
||||
/// Call recursively for lambda expressions.
|
||||
for (size_t i = 0; i < node.arguments->children.size(); ++i)
|
||||
{
|
||||
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
|
||||
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(),
|
||||
getColumnName())));
|
||||
return;
|
||||
}
|
||||
|
||||
if (AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
return;
|
||||
|
||||
/// Context object that we pass to function should live during query.
|
||||
const Context & function_context = context.hasQueryContext()
|
||||
? context.getQueryContext()
|
||||
: context;
|
||||
|
||||
FunctionBuilderPtr function_builder;
|
||||
try
|
||||
{
|
||||
function_builder = FunctionFactory::instance().get(node->name, function_context);
|
||||
}
|
||||
catch (DB::Exception & e)
|
||||
{
|
||||
auto hints = AggregateFunctionFactory::instance().getHints(node->name);
|
||||
if (!hints.empty())
|
||||
e.addMessage("Or unknown aggregate function " + node->name + ". Maybe you meant: " + toString(hints));
|
||||
e.rethrow();
|
||||
}
|
||||
|
||||
Names argument_names;
|
||||
DataTypes argument_types;
|
||||
bool arguments_present = true;
|
||||
|
||||
/// If the function has an argument-lambda expression, you need to determine its type before the recursive call.
|
||||
bool has_lambda_arguments = false;
|
||||
|
||||
for (size_t arg = 0; arg < node->arguments->children.size(); ++arg)
|
||||
{
|
||||
auto & child = node->arguments->children[arg];
|
||||
auto child_column_name = child->getColumnName();
|
||||
ASTPtr child = node.arguments->children[i];
|
||||
|
||||
const auto * lambda = child->as<ASTFunction>();
|
||||
if (lambda && lambda->name == "lambda")
|
||||
{
|
||||
/// If the argument is a lambda expression, just remember its approximate type.
|
||||
if (lambda->arguments->children.size() != 2)
|
||||
throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
|
||||
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||
const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children;
|
||||
NamesAndTypesList lambda_arguments;
|
||||
|
||||
if (!lambda_args_tuple || lambda_args_tuple->name != "tuple")
|
||||
throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
has_lambda_arguments = true;
|
||||
argument_types.emplace_back(std::make_shared<DataTypeFunction>(DataTypes(lambda_args_tuple->arguments->children.size())));
|
||||
/// Select the name in the next cycle.
|
||||
argument_names.emplace_back();
|
||||
}
|
||||
else if (functionIsInOrGlobalInOperator(node->name) && arg == 1 && prepared_set)
|
||||
{
|
||||
ColumnWithTypeAndName column;
|
||||
column.type = std::make_shared<DataTypeSet>();
|
||||
|
||||
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
|
||||
/// so that sets with the same literal representation do not fuse together (they can have different types).
|
||||
if (!prepared_set->empty())
|
||||
column.name = getUniqueName(actions_stack.getSampleBlock(), "__set");
|
||||
else
|
||||
column.name = child_column_name;
|
||||
|
||||
if (!actions_stack.getSampleBlock().has(column.name))
|
||||
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
||||
{
|
||||
column.column = ColumnSet::create(1, prepared_set);
|
||||
auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]);
|
||||
if (!opt_arg_name)
|
||||
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
actions_stack.addAction(ExpressionAction::addColumn(column));
|
||||
lambda_arguments.emplace_back(*opt_arg_name, lambda_type->getArgumentTypes()[j]);
|
||||
}
|
||||
|
||||
argument_types.push_back(column.type);
|
||||
argument_names.push_back(column.name);
|
||||
data.actions_stack.pushLevel(lambda_arguments);
|
||||
visit(lambda->arguments->children.at(1), data);
|
||||
ExpressionActionsPtr lambda_actions = data.actions_stack.popLevel();
|
||||
|
||||
String result_name = lambda->arguments->children.at(1)->getColumnName();
|
||||
lambda_actions->finalize(Names(1, result_name));
|
||||
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
|
||||
|
||||
Names captured;
|
||||
Names required = lambda_actions->getRequiredColumns();
|
||||
for (const auto & required_arg : required)
|
||||
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
|
||||
captured.push_back(required_arg);
|
||||
|
||||
/// We can not name `getColumnName()`,
|
||||
/// because it does not uniquely define the expression (the types of arguments can be different).
|
||||
String lambda_name = getUniqueName(data.getSampleBlock(), "__lambda");
|
||||
|
||||
auto function_capture = std::make_shared<FunctionCapture>(
|
||||
lambda_actions, captured, lambda_arguments, result_type, result_name);
|
||||
data.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name));
|
||||
|
||||
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
|
||||
argument_names[i] = lambda_name;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// If the argument is not a lambda expression, call it recursively and find out its type.
|
||||
visit(child);
|
||||
std::string name = child_column_name;
|
||||
if (actions_stack.getSampleBlock().has(name))
|
||||
{
|
||||
argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type);
|
||||
argument_names.push_back(name);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (only_consts)
|
||||
arguments_present = false;
|
||||
else
|
||||
throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (only_consts && !arguments_present)
|
||||
return;
|
||||
|
||||
if (has_lambda_arguments && !only_consts)
|
||||
{
|
||||
function_builder->getLambdaArgumentTypes(argument_types);
|
||||
|
||||
/// Call recursively for lambda expressions.
|
||||
for (size_t i = 0; i < node->arguments->children.size(); ++i)
|
||||
{
|
||||
ASTPtr child = node->arguments->children[i];
|
||||
|
||||
const auto * lambda = child->as<ASTFunction>();
|
||||
if (lambda && lambda->name == "lambda")
|
||||
{
|
||||
const DataTypeFunction * lambda_type = typeid_cast<const DataTypeFunction *>(argument_types[i].get());
|
||||
const auto * lambda_args_tuple = lambda->arguments->children.at(0)->as<ASTFunction>();
|
||||
const ASTs & lambda_arg_asts = lambda_args_tuple->arguments->children;
|
||||
NamesAndTypesList lambda_arguments;
|
||||
|
||||
for (size_t j = 0; j < lambda_arg_asts.size(); ++j)
|
||||
{
|
||||
auto opt_arg_name = tryGetIdentifierName(lambda_arg_asts[j]);
|
||||
if (!opt_arg_name)
|
||||
throw Exception("lambda argument declarations must be identifiers", ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
lambda_arguments.emplace_back(*opt_arg_name, lambda_type->getArgumentTypes()[j]);
|
||||
}
|
||||
|
||||
actions_stack.pushLevel(lambda_arguments);
|
||||
visit(lambda->arguments->children.at(1));
|
||||
ExpressionActionsPtr lambda_actions = actions_stack.popLevel();
|
||||
|
||||
String result_name = lambda->arguments->children.at(1)->getColumnName();
|
||||
lambda_actions->finalize(Names(1, result_name));
|
||||
DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type;
|
||||
|
||||
Names captured;
|
||||
Names required = lambda_actions->getRequiredColumns();
|
||||
for (const auto & required_arg : required)
|
||||
if (findColumn(required_arg, lambda_arguments) == lambda_arguments.end())
|
||||
captured.push_back(required_arg);
|
||||
|
||||
/// We can not name `getColumnName()`,
|
||||
/// because it does not uniquely define the expression (the types of arguments can be different).
|
||||
String lambda_name = getUniqueName(actions_stack.getSampleBlock(), "__lambda");
|
||||
|
||||
auto function_capture = std::make_shared<FunctionCapture>(
|
||||
lambda_actions, captured, lambda_arguments, result_type, result_name);
|
||||
actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name));
|
||||
|
||||
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
|
||||
argument_names[i] = lambda_name;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (only_consts)
|
||||
{
|
||||
for (const auto & argument_name : argument_names)
|
||||
{
|
||||
if (!actions_stack.getSampleBlock().has(argument_name))
|
||||
{
|
||||
arguments_present = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (arguments_present)
|
||||
{
|
||||
actions_stack.addAction(
|
||||
ExpressionAction::applyFunction(function_builder, argument_names, getColumnName()));
|
||||
}
|
||||
}
|
||||
else if (const auto * literal = ast->as<ASTLiteral>())
|
||||
{
|
||||
DataTypePtr type = applyVisitor(FieldToDataType(), literal->value);
|
||||
|
||||
ColumnWithTypeAndName column;
|
||||
column.column = type->createColumnConst(1, convertFieldToType(literal->value, *type));
|
||||
column.type = type;
|
||||
column.name = getColumnName();
|
||||
|
||||
actions_stack.addAction(ExpressionAction::addColumn(column));
|
||||
}
|
||||
else
|
||||
if (data.only_consts)
|
||||
{
|
||||
for (auto & child : ast->children)
|
||||
for (const auto & argument_name : argument_names)
|
||||
{
|
||||
/// Do not go to FROM, JOIN, UNION.
|
||||
if (!child->as<ASTTableExpression>() && !child->as<ASTSelectQuery>())
|
||||
visit(child);
|
||||
if (!data.hasColumn(argument_name))
|
||||
{
|
||||
arguments_present = false;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (arguments_present)
|
||||
{
|
||||
data.addAction(ExpressionAction::applyFunction(function_builder, argument_names, column_name.get(ast)));
|
||||
}
|
||||
}
|
||||
|
||||
SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_block)
|
||||
void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data)
|
||||
{
|
||||
CachedColumnName column_name;
|
||||
if (data.hasColumn(column_name.get(ast)))
|
||||
return;
|
||||
|
||||
DataTypePtr type = applyVisitor(FieldToDataType(), literal.value);
|
||||
|
||||
ColumnWithTypeAndName column;
|
||||
column.column = type->createColumnConst(1, convertFieldToType(literal.value, *type));
|
||||
column.type = type;
|
||||
column.name = column_name.get(ast);
|
||||
|
||||
data.addAction(ExpressionAction::addColumn(column));
|
||||
}
|
||||
|
||||
SetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data)
|
||||
{
|
||||
/** You need to convert the right argument to a set.
|
||||
* This can be a table name, a value, a value enumeration, or a subquery.
|
||||
* The enumeration of values is parsed as a function `tuple`.
|
||||
*/
|
||||
const IAST & args = *node->arguments;
|
||||
const IAST & args = *node.arguments;
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
const Block & sample_block = data.getSampleBlock();
|
||||
|
||||
/// If the subquery or table name for SELECT.
|
||||
const auto * identifier = arg->as<ASTIdentifier>();
|
||||
if (arg->as<ASTSubquery>() || identifier)
|
||||
{
|
||||
auto set_key = PreparedSetKey::forSubquery(*arg);
|
||||
if (prepared_sets.count(set_key))
|
||||
return prepared_sets.at(set_key);
|
||||
if (data.prepared_sets.count(set_key))
|
||||
return data.prepared_sets.at(set_key);
|
||||
|
||||
/// A special case is if the name of the table is specified on the right side of the IN statement,
|
||||
/// and the table has the type Set (a previously prepared set).
|
||||
if (identifier)
|
||||
{
|
||||
DatabaseAndTableWithAlias database_table(*identifier);
|
||||
StoragePtr table = context.tryGetTable(database_table.database, database_table.table);
|
||||
StoragePtr table = data.context.tryGetTable(database_table.database, database_table.table);
|
||||
|
||||
if (table)
|
||||
{
|
||||
StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get());
|
||||
if (storage_set)
|
||||
{
|
||||
prepared_sets[set_key] = storage_set->getSet();
|
||||
data.prepared_sets[set_key] = storage_set->getSet();
|
||||
return storage_set->getSet();
|
||||
}
|
||||
}
|
||||
@ -576,25 +580,25 @@ SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bl
|
||||
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
|
||||
String set_id = arg->getColumnName();
|
||||
|
||||
SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id];
|
||||
SubqueryForSet & subquery_for_set = data.subqueries_for_sets[set_id];
|
||||
|
||||
/// If you already created a Set with the same subquery / table.
|
||||
if (subquery_for_set.set)
|
||||
{
|
||||
prepared_sets[set_key] = subquery_for_set.set;
|
||||
data.prepared_sets[set_key] = subquery_for_set.set;
|
||||
return subquery_for_set.set;
|
||||
}
|
||||
|
||||
SetPtr set = std::make_shared<Set>(set_size_limit, false);
|
||||
SetPtr set = std::make_shared<Set>(data.set_size_limit, false);
|
||||
|
||||
/** The following happens for GLOBAL INs:
|
||||
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
|
||||
* in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table.
|
||||
* - this function shows the expression IN_data1.
|
||||
*/
|
||||
if (!subquery_for_set.source && no_storage_or_local)
|
||||
if (!subquery_for_set.source && data.no_storage_or_local)
|
||||
{
|
||||
auto interpreter = interpretSubquery(arg, context, subquery_depth, {});
|
||||
auto interpreter = interpretSubquery(arg, data.context, data.subquery_depth, {});
|
||||
subquery_for_set.source = std::make_shared<LazyBlockInputStream>(
|
||||
interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; });
|
||||
|
||||
@ -627,13 +631,13 @@ SetPtr ActionsVisitor::makeSet(const ASTFunction * node, const Block & sample_bl
|
||||
}
|
||||
|
||||
subquery_for_set.set = set;
|
||||
prepared_sets[set_key] = set;
|
||||
data.prepared_sets[set_key] = set;
|
||||
return set;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// An explicit enumeration of values in parentheses.
|
||||
return makeExplicitSet(node, sample_block, false, context, set_size_limit, prepared_sets);
|
||||
return makeExplicitSet(&node, sample_block, false, data.context, data.set_size_limit, data.prepared_sets);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/PreparedSets.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Interpreters/SubqueryForSet.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -52,36 +53,81 @@ struct ScopeStack
|
||||
const Block & getSampleBlock() const;
|
||||
};
|
||||
|
||||
class ASTIdentifier;
|
||||
class ASTFunction;
|
||||
class ASTLiteral;
|
||||
|
||||
/// Collect ExpressionAction from AST. Returns PreparedSets and SubqueriesForSets too.
|
||||
/// After AST is visited source ExpressionActions should be updated with popActionsLevel() method.
|
||||
class ActionsVisitor
|
||||
class ActionsMatcher
|
||||
{
|
||||
public:
|
||||
ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
|
||||
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr);
|
||||
using Visitor = ConstInDepthNodeVisitor<ActionsMatcher, true>;
|
||||
|
||||
void visit(const ASTPtr & ast);
|
||||
struct Data
|
||||
{
|
||||
const Context & context;
|
||||
SizeLimits set_size_limit;
|
||||
size_t subquery_depth;
|
||||
const NamesAndTypesList & source_columns;
|
||||
PreparedSets & prepared_sets;
|
||||
SubqueriesForSets & subqueries_for_sets;
|
||||
bool no_subqueries;
|
||||
bool only_consts;
|
||||
bool no_storage_or_local;
|
||||
size_t visit_depth;
|
||||
ScopeStack actions_stack;
|
||||
|
||||
ExpressionActionsPtr popActionsLevel() { return actions_stack.popLevel(); }
|
||||
Data(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_,
|
||||
const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions,
|
||||
PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_,
|
||||
bool no_subqueries_, bool only_consts_, bool no_storage_or_local_)
|
||||
: context(context_),
|
||||
set_size_limit(set_size_limit_),
|
||||
subquery_depth(subquery_depth_),
|
||||
source_columns(source_columns_),
|
||||
prepared_sets(prepared_sets_),
|
||||
subqueries_for_sets(subqueries_for_sets_),
|
||||
no_subqueries(no_subqueries_),
|
||||
only_consts(only_consts_),
|
||||
no_storage_or_local(no_storage_or_local_),
|
||||
visit_depth(0),
|
||||
actions_stack(actions, context)
|
||||
{}
|
||||
|
||||
void updateActions(ExpressionActionsPtr & actions)
|
||||
{
|
||||
actions = actions_stack.popLevel();
|
||||
}
|
||||
|
||||
void addAction(const ExpressionAction & action)
|
||||
{
|
||||
actions_stack.addAction(action);
|
||||
}
|
||||
|
||||
const Block & getSampleBlock() const
|
||||
{
|
||||
return actions_stack.getSampleBlock();
|
||||
}
|
||||
|
||||
/// Does result of the calculation already exists in the block.
|
||||
bool hasColumn(const String & columnName) const
|
||||
{
|
||||
return actions_stack.getSampleBlock().has(columnName);
|
||||
}
|
||||
};
|
||||
|
||||
static void visit(const ASTPtr & ast, Data & data);
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
SizeLimits set_size_limit;
|
||||
size_t subquery_depth;
|
||||
const NamesAndTypesList & source_columns;
|
||||
PreparedSets & prepared_sets;
|
||||
SubqueriesForSets & subqueries_for_sets;
|
||||
const bool no_subqueries;
|
||||
const bool only_consts;
|
||||
const bool no_storage_or_local;
|
||||
mutable size_t visit_depth;
|
||||
std::ostream * ostr;
|
||||
ScopeStack actions_stack;
|
||||
|
||||
SetPtr makeSet(const ASTFunction * node, const Block & sample_block);
|
||||
static void visit(const ASTIdentifier & identifier, const ASTPtr & ast, Data & data);
|
||||
static void visit(const ASTFunction & node, const ASTPtr & ast, Data & data);
|
||||
static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data);
|
||||
|
||||
static SetPtr makeSet(const ASTFunction & node, Data & data);
|
||||
};
|
||||
|
||||
using ActionsVisitor = ActionsMatcher::Visitor;
|
||||
|
||||
}
|
||||
|
@ -11,7 +11,6 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <AggregateFunctions/AggregateFunctionCount.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/NativeBlockOutputStream.h>
|
||||
#include <DataStreams/NullBlockInputStream.h>
|
||||
@ -289,7 +288,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
"template void Aggregator::executeSpecialized<\n"
|
||||
" " << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n"
|
||||
" " << method_typename << " &, Arena *, size_t, ColumnRawPtrs &,\n"
|
||||
" AggregateColumns &, StringRefs &, bool, AggregateDataPtr) const;\n"
|
||||
" AggregateColumns &, bool, AggregateDataPtr) const;\n"
|
||||
"\n"
|
||||
"static void wrapper" << suffix << "(\n"
|
||||
" const Aggregator & aggregator,\n"
|
||||
@ -298,13 +297,12 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
" size_t rows,\n"
|
||||
" ColumnRawPtrs & key_columns,\n"
|
||||
" Aggregator::AggregateColumns & aggregate_columns,\n"
|
||||
" StringRefs & keys,\n"
|
||||
" bool no_more_keys,\n"
|
||||
" AggregateDataPtr overflow_row)\n"
|
||||
"{\n"
|
||||
" aggregator.executeSpecialized<\n"
|
||||
" " << method_typename << ", TypeList<" << aggregate_functions_typenames << ">>(\n"
|
||||
" method, arena, rows, key_columns, aggregate_columns, keys, no_more_keys, overflow_row);\n"
|
||||
" method, arena, rows, key_columns, aggregate_columns, no_more_keys, overflow_row);\n"
|
||||
"}\n"
|
||||
"\n"
|
||||
"void * getPtr" << suffix << "() __attribute__((__visibility__(\"default\")));\n"
|
||||
@ -583,16 +581,16 @@ void NO_INLINE Aggregator::executeImpl(
|
||||
size_t rows,
|
||||
ColumnRawPtrs & key_columns,
|
||||
AggregateFunctionInstruction * aggregate_instructions,
|
||||
StringRefs & keys,
|
||||
bool no_more_keys,
|
||||
AggregateDataPtr overflow_row) const
|
||||
{
|
||||
typename Method::State state(key_columns, key_sizes, aggregation_state_cache);
|
||||
|
||||
if (!no_more_keys)
|
||||
executeImplCase<false>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, keys, overflow_row);
|
||||
//executeImplCase<false>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row);
|
||||
executeImplBatch(method, state, aggregates_pool, rows, aggregate_instructions);
|
||||
else
|
||||
executeImplCase<true>(method, state, aggregates_pool, rows, key_columns, aggregate_instructions, keys, overflow_row);
|
||||
executeImplCase<true>(method, state, aggregates_pool, rows, aggregate_instructions, overflow_row);
|
||||
}
|
||||
|
||||
|
||||
@ -602,9 +600,7 @@ void NO_INLINE Aggregator::executeImplCase(
|
||||
typename Method::State & state,
|
||||
Arena * aggregates_pool,
|
||||
size_t rows,
|
||||
ColumnRawPtrs & /*key_columns*/,
|
||||
AggregateFunctionInstruction * aggregate_instructions,
|
||||
StringRefs & /*keys*/,
|
||||
AggregateDataPtr overflow_row) const
|
||||
{
|
||||
/// NOTE When editing this code, also pay attention to SpecializedAggregator.h.
|
||||
@ -655,34 +651,60 @@ void NO_INLINE Aggregator::executeImplCase(
|
||||
}
|
||||
|
||||
|
||||
template <typename Method>
|
||||
void NO_INLINE Aggregator::executeImplBatch(
|
||||
Method & method,
|
||||
typename Method::State & state,
|
||||
Arena * aggregates_pool,
|
||||
size_t rows,
|
||||
AggregateFunctionInstruction * aggregate_instructions) const
|
||||
{
|
||||
PODArray<AggregateDataPtr> places(rows);
|
||||
|
||||
/// For all rows.
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
AggregateDataPtr aggregate_data = nullptr;
|
||||
|
||||
auto emplace_result = state.emplaceKey(method.data, i, *aggregates_pool);
|
||||
|
||||
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
|
||||
if (emplace_result.isInserted())
|
||||
{
|
||||
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
|
||||
emplace_result.setMapped(nullptr);
|
||||
|
||||
aggregate_data = aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states);
|
||||
createAggregateStates(aggregate_data);
|
||||
|
||||
emplace_result.setMapped(aggregate_data);
|
||||
}
|
||||
else
|
||||
aggregate_data = emplace_result.getMapped();
|
||||
|
||||
places[i] = aggregate_data;
|
||||
}
|
||||
|
||||
/// Add values to the aggregate functions.
|
||||
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
||||
inst->that->addBatch(rows, places.data(), inst->state_offset, inst->arguments, aggregates_pool);
|
||||
}
|
||||
|
||||
|
||||
void NO_INLINE Aggregator::executeWithoutKeyImpl(
|
||||
AggregatedDataWithoutKey & res,
|
||||
size_t rows,
|
||||
AggregateFunctionInstruction * aggregate_instructions,
|
||||
Arena * arena) const
|
||||
{
|
||||
/// Optimization in the case of a single aggregate function `count`.
|
||||
AggregateFunctionCount * agg_count = params.aggregates_size == 1
|
||||
? typeid_cast<AggregateFunctionCount *>(aggregate_functions[0])
|
||||
: nullptr;
|
||||
|
||||
if (agg_count)
|
||||
agg_count->addDelta(res, rows);
|
||||
else
|
||||
{
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
/// Adding values
|
||||
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
||||
(*inst->func)(inst->that, res + inst->state_offset, inst->arguments, i, arena);
|
||||
}
|
||||
}
|
||||
/// Adding values
|
||||
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
||||
inst->that->addBatchSinglePlace(rows, res + inst->state_offset, inst->arguments, arena);
|
||||
}
|
||||
|
||||
|
||||
bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & result,
|
||||
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, StringRefs & key,
|
||||
bool & no_more_keys)
|
||||
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys)
|
||||
{
|
||||
if (isCancelled())
|
||||
return true;
|
||||
@ -796,9 +818,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
|
||||
reinterpret_cast<void (*)( \
|
||||
const Aggregator &, decltype(result.NAME)::element_type &, \
|
||||
Arena *, size_t, ColumnRawPtrs &, AggregateColumns &, \
|
||||
StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_method_ptr) \
|
||||
bool, AggregateDataPtr)>(compiled_data->compiled_method_ptr) \
|
||||
(*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \
|
||||
key, no_more_keys, overflow_row_ptr);
|
||||
no_more_keys, overflow_row_ptr);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
@ -812,9 +834,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
|
||||
reinterpret_cast<void (*)( \
|
||||
const Aggregator &, decltype(result.NAME)::element_type &, \
|
||||
Arena *, size_t, ColumnRawPtrs &, AggregateColumns &, \
|
||||
StringRefs &, bool, AggregateDataPtr)>(compiled_data->compiled_two_level_method_ptr) \
|
||||
bool, AggregateDataPtr)>(compiled_data->compiled_two_level_method_ptr) \
|
||||
(*this, *result.NAME, result.aggregates_pool, rows, key_columns, aggregate_columns, \
|
||||
key, no_more_keys, overflow_row_ptr);
|
||||
no_more_keys, overflow_row_ptr);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
||||
@ -826,7 +848,7 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
else if (result.type == AggregatedDataVariants::Type::NAME) \
|
||||
executeImpl(*result.NAME, result.aggregates_pool, rows, key_columns, aggregate_functions_instructions.data(), \
|
||||
key, no_more_keys, overflow_row_ptr);
|
||||
no_more_keys, overflow_row_ptr);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
@ -1032,7 +1054,6 @@ void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVaria
|
||||
if (isCancelled())
|
||||
return;
|
||||
|
||||
StringRefs key(params.keys_size);
|
||||
ColumnRawPtrs key_columns(params.keys_size);
|
||||
AggregateColumns aggregate_columns(params.aggregates_size);
|
||||
|
||||
@ -1059,14 +1080,14 @@ void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVaria
|
||||
src_rows += block.rows();
|
||||
src_bytes += block.bytes();
|
||||
|
||||
if (!executeOnBlock(block, result, key_columns, aggregate_columns, key, no_more_keys))
|
||||
if (!executeOnBlock(block, result, key_columns, aggregate_columns, no_more_keys))
|
||||
break;
|
||||
}
|
||||
|
||||
/// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation.
|
||||
/// To do this, we pass a block with zero rows to aggregate.
|
||||
if (result.empty() && params.keys_size == 0 && !params.empty_result_for_aggregation_by_empty_set)
|
||||
executeOnBlock(stream->getHeader(), result, key_columns, aggregate_columns, key, no_more_keys);
|
||||
executeOnBlock(stream->getHeader(), result, key_columns, aggregate_columns, no_more_keys);
|
||||
|
||||
double elapsed_seconds = watch.elapsedSeconds();
|
||||
size_t rows = result.sizeWithoutOverflowRow();
|
||||
@ -2321,7 +2342,6 @@ void NO_INLINE Aggregator::convertBlockToTwoLevelImpl(
|
||||
Method & method,
|
||||
Arena * pool,
|
||||
ColumnRawPtrs & key_columns,
|
||||
StringRefs & keys [[maybe_unused]],
|
||||
const Block & source,
|
||||
std::vector<Block> & destinations) const
|
||||
{
|
||||
@ -2383,7 +2403,6 @@ std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
|
||||
|
||||
AggregatedDataVariants data;
|
||||
|
||||
StringRefs key(params.keys_size);
|
||||
ColumnRawPtrs key_columns(params.keys_size);
|
||||
|
||||
/// Remember the columns we will work with
|
||||
@ -2423,7 +2442,7 @@ std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
|
||||
#define M(NAME) \
|
||||
else if (data.type == AggregatedDataVariants::Type::NAME) \
|
||||
convertBlockToTwoLevelImpl(*data.NAME, data.aggregates_pool, \
|
||||
key_columns, key, block, splitted_blocks);
|
||||
key_columns, block, splitted_blocks);
|
||||
|
||||
if (false) {}
|
||||
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
||||
|
@ -842,7 +842,6 @@ public:
|
||||
/// Process one block. Return false if the processing should be aborted (with group_by_overflow_mode = 'break').
|
||||
bool executeOnBlock(const Block & block, AggregatedDataVariants & result,
|
||||
ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block
|
||||
StringRefs & keys, /// - pass the corresponding objects that are initially empty.
|
||||
bool & no_more_keys);
|
||||
|
||||
/** Convert the aggregation data structure into a block.
|
||||
@ -1003,7 +1002,6 @@ protected:
|
||||
size_t rows,
|
||||
ColumnRawPtrs & key_columns,
|
||||
AggregateFunctionInstruction * aggregate_instructions,
|
||||
StringRefs & keys,
|
||||
bool no_more_keys,
|
||||
AggregateDataPtr overflow_row) const;
|
||||
|
||||
@ -1014,11 +1012,17 @@ protected:
|
||||
typename Method::State & state,
|
||||
Arena * aggregates_pool,
|
||||
size_t rows,
|
||||
ColumnRawPtrs & key_columns,
|
||||
AggregateFunctionInstruction * aggregate_instructions,
|
||||
StringRefs & keys,
|
||||
AggregateDataPtr overflow_row) const;
|
||||
|
||||
template <typename Method>
|
||||
void executeImplBatch(
|
||||
Method & method,
|
||||
typename Method::State & state,
|
||||
Arena * aggregates_pool,
|
||||
size_t rows,
|
||||
AggregateFunctionInstruction * aggregate_instructions) const;
|
||||
|
||||
/// For case when there are no keys (all aggregate into one row).
|
||||
void executeWithoutKeyImpl(
|
||||
AggregatedDataWithoutKey & res,
|
||||
@ -1042,7 +1046,6 @@ public:
|
||||
size_t rows,
|
||||
ColumnRawPtrs & key_columns,
|
||||
AggregateColumns & aggregate_columns,
|
||||
StringRefs & keys,
|
||||
bool no_more_keys,
|
||||
AggregateDataPtr overflow_row) const;
|
||||
|
||||
@ -1052,9 +1055,7 @@ public:
|
||||
typename Method::State & state,
|
||||
Arena * aggregates_pool,
|
||||
size_t rows,
|
||||
ColumnRawPtrs & key_columns,
|
||||
AggregateColumns & aggregate_columns,
|
||||
StringRefs & keys,
|
||||
AggregateDataPtr overflow_row) const;
|
||||
|
||||
template <typename AggregateFunctionsList>
|
||||
@ -1179,7 +1180,6 @@ protected:
|
||||
Method & method,
|
||||
Arena * pool,
|
||||
ColumnRawPtrs & key_columns,
|
||||
StringRefs & keys,
|
||||
const Block & source,
|
||||
std::vector<Block> & destinations) const;
|
||||
|
||||
|
@ -1,7 +1,5 @@
|
||||
#include <Interpreters/AnalyzedJoin.h>
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/ExpressionAnalyzer.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
@ -50,40 +48,6 @@ size_t AnalyzedJoin::rightKeyInclusion(const String & name) const
|
||||
return count;
|
||||
}
|
||||
|
||||
ExpressionActionsPtr AnalyzedJoin::createJoinedBlockActions(
|
||||
const NamesAndTypesList & columns_added_by_join,
|
||||
const ASTSelectQuery * select_query_with_join,
|
||||
const Context & context) const
|
||||
{
|
||||
if (!select_query_with_join)
|
||||
return nullptr;
|
||||
|
||||
const ASTTablesInSelectQueryElement * join = select_query_with_join->join();
|
||||
|
||||
if (!join)
|
||||
return nullptr;
|
||||
|
||||
const auto & join_params = join->table_join->as<ASTTableJoin &>();
|
||||
|
||||
/// Create custom expression list with join keys from right table.
|
||||
auto expression_list = std::make_shared<ASTExpressionList>();
|
||||
ASTs & children = expression_list->children;
|
||||
|
||||
if (join_params.on_expression)
|
||||
for (const auto & join_right_key : key_asts_right)
|
||||
children.emplace_back(join_right_key);
|
||||
|
||||
NameSet required_columns_set(key_names_right.begin(), key_names_right.end());
|
||||
for (const auto & joined_column : columns_added_by_join)
|
||||
required_columns_set.insert(joined_column.name);
|
||||
Names required_columns(required_columns_set.begin(), required_columns_set.end());
|
||||
|
||||
ASTPtr query = expression_list;
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query, columns_from_joined_table, required_columns);
|
||||
ExpressionAnalyzer analyzer(query, syntax_result, context, required_columns_set);
|
||||
return analyzer.getActions(true, false);
|
||||
}
|
||||
|
||||
void AnalyzedJoin::deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix)
|
||||
{
|
||||
NameSet joined_columns;
|
||||
|
@ -14,9 +14,6 @@ class Context;
|
||||
class ASTSelectQuery;
|
||||
struct DatabaseAndTableWithAlias;
|
||||
|
||||
class ExpressionActions;
|
||||
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
|
||||
|
||||
struct AnalyzedJoin
|
||||
{
|
||||
/** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
|
||||
@ -35,6 +32,7 @@ private:
|
||||
friend class SyntaxAnalyzer;
|
||||
friend struct SyntaxAnalyzerResult;
|
||||
friend class ExpressionAnalyzer;
|
||||
friend class SelectQueryExpressionAnalyzer;
|
||||
|
||||
Names key_names_left;
|
||||
Names key_names_right; /// Duplicating names are qualified.
|
||||
@ -55,10 +53,8 @@ public:
|
||||
void addUsingKey(const ASTPtr & ast);
|
||||
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
|
||||
|
||||
ExpressionActionsPtr createJoinedBlockActions(
|
||||
const NamesAndTypesList & columns_added_by_join, /// Subset of available_joined_columns.
|
||||
const ASTSelectQuery * select_query_with_join,
|
||||
const Context & context) const;
|
||||
bool hasUsing() const { return with_using; }
|
||||
bool hasOn() const { return !with_using; }
|
||||
|
||||
NameSet getQualifiedColumnsSet() const;
|
||||
NameSet getOriginalColumnsSet() const;
|
||||
|
@ -180,19 +180,22 @@ void AsynchronousMetrics::update()
|
||||
calculateMaxAndSum(max_inserts_in_queue, sum_inserts_in_queue, status.queue.inserts_in_queue);
|
||||
calculateMaxAndSum(max_merges_in_queue, sum_merges_in_queue, status.queue.merges_in_queue);
|
||||
|
||||
try
|
||||
if (!status.is_readonly)
|
||||
{
|
||||
time_t absolute_delay = 0;
|
||||
time_t relative_delay = 0;
|
||||
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
|
||||
try
|
||||
{
|
||||
time_t absolute_delay = 0;
|
||||
time_t relative_delay = 0;
|
||||
table_replicated_merge_tree->getReplicaDelays(absolute_delay, relative_delay);
|
||||
|
||||
calculateMax(max_absolute_delay, absolute_delay);
|
||||
calculateMax(max_relative_delay, relative_delay);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__,
|
||||
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
|
||||
calculateMax(max_absolute_delay, absolute_delay);
|
||||
calculateMax(max_relative_delay, relative_delay);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__,
|
||||
"Cannot get replica delay for table: " + backQuoteIfNeed(db.first) + "." + backQuoteIfNeed(iterator->name()));
|
||||
}
|
||||
}
|
||||
|
||||
calculateMax(max_part_count_for_partition, table_replicated_merge_tree->getMaxPartsCountForPartition());
|
||||
|
@ -38,10 +38,6 @@
|
||||
#include <Interpreters/Compiler.h>
|
||||
#include <Interpreters/SettingsConstraints.h>
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Interpreters/QueryLog.h>
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/DDLWorker.h>
|
||||
#include <Common/DNSResolver.h>
|
||||
@ -659,6 +655,10 @@ void Context::setProfile(const String & profile)
|
||||
settings_constraints = std::move(new_constraints);
|
||||
}
|
||||
|
||||
std::shared_ptr<const User> Context::getUser(const String & user_name)
|
||||
{
|
||||
return shared->users_manager->getUser(user_name);
|
||||
}
|
||||
|
||||
void Context::setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key)
|
||||
{
|
||||
@ -1701,6 +1701,7 @@ std::shared_ptr<PartLog> Context::getPartLog(const String & part_database)
|
||||
return shared->system_logs->part_log;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<TraceLog> Context::getTraceLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -1711,6 +1712,7 @@ std::shared_ptr<TraceLog> Context::getTraceLog()
|
||||
return shared->system_logs->trace_log;
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<TextLog> Context::getTextLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
@ -1722,6 +1724,17 @@ std::shared_ptr<TextLog> Context::getTextLog()
|
||||
}
|
||||
|
||||
|
||||
std::shared_ptr<MetricLog> Context::getMetricLog()
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
if (!shared->system_logs || !shared->system_logs->metric_log)
|
||||
return {};
|
||||
|
||||
return shared->system_logs->metric_log;
|
||||
}
|
||||
|
||||
|
||||
CompressionCodecPtr Context::chooseCompressionCodec(size_t part_size, double part_size_ratio) const
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <DataStreams/IBlockStream_fwd.h>
|
||||
#include <Interpreters/ClientInfo.h>
|
||||
#include <Interpreters/Users.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Common/LRUCache.h>
|
||||
#include <Common/MultiVersion.h>
|
||||
@ -64,6 +65,7 @@ class QueryThreadLog;
|
||||
class PartLog;
|
||||
class TextLog;
|
||||
class TraceLog;
|
||||
class MetricLog;
|
||||
struct MergeTreeSettings;
|
||||
class IDatabase;
|
||||
class DDLGuard;
|
||||
@ -199,6 +201,10 @@ public:
|
||||
|
||||
/// Must be called before getClientInfo.
|
||||
void setUser(const String & name, const String & password, const Poco::Net::SocketAddress & address, const String & quota_key);
|
||||
|
||||
/// Used by MySQL Secure Password Authentication plugin.
|
||||
std::shared_ptr<const User> getUser(const String & user_name);
|
||||
|
||||
/// Compute and set actual user settings, client_info.current_user should be set
|
||||
void calculateUserSettings();
|
||||
|
||||
@ -434,6 +440,7 @@ public:
|
||||
std::shared_ptr<QueryThreadLog> getQueryThreadLog();
|
||||
std::shared_ptr<TraceLog> getTraceLog();
|
||||
std::shared_ptr<TextLog> getTextLog();
|
||||
std::shared_ptr<MetricLog> getMetricLog();
|
||||
|
||||
/// Returns an object used to log opertaions with parts if it possible.
|
||||
/// Provide table name to make required cheks.
|
||||
|
@ -55,8 +55,11 @@
|
||||
#include <Interpreters/DatabaseAndTableWithAlias.h>
|
||||
#include <Interpreters/QueryNormalizer.h>
|
||||
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
|
||||
#include <Interpreters/ExternalTablesVisitor.h>
|
||||
#include <Interpreters/GlobalSubqueriesVisitor.h>
|
||||
#include <Interpreters/GetAggregatesVisitor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -67,26 +70,22 @@ using LogAST = DebugASTLog<false>; /// set to true to enable logs
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
extern const int ILLEGAL_AGGREGATION;
|
||||
extern const int EXPECTED_ALL_OR_ANY;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
ExpressionAnalyzer::ExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
const NameSet & required_result_columns_,
|
||||
size_t subquery_depth_,
|
||||
bool do_global_,
|
||||
const SubqueriesForSets & subqueries_for_sets_)
|
||||
: ExpressionAnalyzerData(required_result_columns_, subqueries_for_sets_)
|
||||
, query(query_), context(context_), settings(context.getSettings())
|
||||
, subquery_depth(subquery_depth_), do_global(do_global_)
|
||||
bool do_global)
|
||||
: query(query_), context(context_), settings(context.getSettings())
|
||||
, subquery_depth(subquery_depth_)
|
||||
, syntax(syntax_analyzer_result_)
|
||||
{
|
||||
/// external_tables, subqueries_for_sets for global subqueries.
|
||||
/// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers.
|
||||
initGlobalSubqueriesAndExternalTables();
|
||||
initGlobalSubqueriesAndExternalTables(do_global);
|
||||
|
||||
/// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns.
|
||||
/// This analysis should be performed after processing global subqueries, because otherwise,
|
||||
@ -115,9 +114,6 @@ void ExpressionAnalyzer::analyzeAggregation()
|
||||
|
||||
auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
if (select_query && (select_query->groupBy() || select_query->having()))
|
||||
has_aggregation = true;
|
||||
|
||||
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(sourceColumns(), context);
|
||||
|
||||
if (select_query)
|
||||
@ -145,15 +141,17 @@ void ExpressionAnalyzer::analyzeAggregation()
|
||||
for (const auto & key_ast : analyzedJoin().key_asts_left)
|
||||
getRootActions(key_ast, true, temp_actions);
|
||||
|
||||
addJoinAction(temp_actions, true);
|
||||
addJoinAction(temp_actions);
|
||||
}
|
||||
}
|
||||
|
||||
getAggregates(query, temp_actions);
|
||||
has_aggregation = makeAggregateDescriptions(temp_actions);
|
||||
if (select_query && (select_query->groupBy() || select_query->having()))
|
||||
has_aggregation = true;
|
||||
|
||||
if (has_aggregation)
|
||||
{
|
||||
assertSelect();
|
||||
getSelectQuery(); /// assertSelect()
|
||||
|
||||
/// Find out aggregation keys.
|
||||
if (select_query->groupBy())
|
||||
@ -222,7 +220,7 @@ void ExpressionAnalyzer::analyzeAggregation()
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables()
|
||||
void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global)
|
||||
{
|
||||
/// Adds existing external tables (not subqueries) to the external_tables dictionary.
|
||||
ExternalTablesVisitor::Data tables_data{context, external_tables};
|
||||
@ -237,21 +235,7 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables()
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::makeSetsForIndex()
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
if (storage() && select_query && storage()->supportsIndexForIn())
|
||||
{
|
||||
if (select_query->where())
|
||||
makeSetsForIndexImpl(select_query->where());
|
||||
if (select_query->prewhere())
|
||||
makeSetsForIndexImpl(select_query->prewhere());
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
|
||||
void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name)
|
||||
{
|
||||
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
|
||||
if (prepared_sets.count(set_key))
|
||||
@ -276,8 +260,12 @@ void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
|
||||
/// Perfomance optimisation for IN() if storage supports it.
|
||||
void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
|
||||
{
|
||||
if (!node || !storage() || !storage()->supportsIndexForIn())
|
||||
return;
|
||||
|
||||
for (auto & child : node->children)
|
||||
{
|
||||
/// Don't descend into subqueries.
|
||||
@ -289,15 +277,16 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
|
||||
if (func && func->name == "lambda")
|
||||
continue;
|
||||
|
||||
makeSetsForIndexImpl(child);
|
||||
makeSetsForIndex(child);
|
||||
}
|
||||
|
||||
const auto * func = node->as<ASTFunction>();
|
||||
if (func && functionIsInOperator(func->name))
|
||||
{
|
||||
const IAST & args = *func->arguments;
|
||||
const ASTPtr & left_in_operand = args.children.at(0);
|
||||
|
||||
if (storage() && storage()->mayBenefitFromIndexForIn(args.children.at(0), context))
|
||||
if (storage()->mayBenefitFromIndexForIn(left_in_operand, context))
|
||||
{
|
||||
const ASTPtr & arg = args.children.at(1);
|
||||
if (arg->as<ASTSubquery>() || arg->as<ASTIdentifier>())
|
||||
@ -309,13 +298,13 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
|
||||
{
|
||||
NamesAndTypesList temp_columns = sourceColumns();
|
||||
temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end());
|
||||
for (const auto & joined_column : columnsAddedByJoin())
|
||||
temp_columns.push_back(joined_column);
|
||||
temp_columns.insert(temp_columns.end(), columnsAddedByJoin().begin(), columnsAddedByJoin().end());
|
||||
|
||||
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, context);
|
||||
getRootActions(func->arguments->children.at(0), true, temp_actions);
|
||||
getRootActions(left_in_operand, true, temp_actions);
|
||||
|
||||
Block sample_block_with_calculated_columns = temp_actions->getSampleBlock();
|
||||
if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName()))
|
||||
if (sample_block_with_calculated_columns.has(left_in_operand->getColumnName()))
|
||||
makeExplicitSet(func, sample_block_with_calculated_columns, true, context,
|
||||
settings.size_limits_for_set, prepared_sets);
|
||||
}
|
||||
@ -327,74 +316,27 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node)
|
||||
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts)
|
||||
{
|
||||
LogAST log;
|
||||
ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth,
|
||||
ActionsVisitor::Data visitor_data(context, settings.size_limits_for_set, subquery_depth,
|
||||
sourceColumns(), actions, prepared_sets, subqueries_for_sets,
|
||||
no_subqueries, only_consts, !isRemoteStorage(), log.stream());
|
||||
actions_visitor.visit(ast);
|
||||
actions = actions_visitor.popActionsLevel();
|
||||
no_subqueries, only_consts, !isRemoteStorage());
|
||||
ActionsVisitor(visitor_data, log.stream()).visit(ast);
|
||||
visitor_data.updateActions(actions);
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions)
|
||||
bool ExpressionAnalyzer::makeAggregateDescriptions(ExpressionActionsPtr & actions)
|
||||
{
|
||||
bool only_consts = false;
|
||||
|
||||
LogAST log;
|
||||
ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth,
|
||||
sourceColumns(), actions, prepared_sets, subqueries_for_sets,
|
||||
no_subqueries, only_consts, !isRemoteStorage(), log.stream());
|
||||
|
||||
if (table_join.using_expression_list)
|
||||
actions_visitor.visit(table_join.using_expression_list);
|
||||
else if (table_join.on_expression)
|
||||
for (const ASTFunction * node : aggregates())
|
||||
{
|
||||
for (const auto & ast : analyzedJoin().key_asts_left)
|
||||
actions_visitor.visit(ast);
|
||||
}
|
||||
|
||||
actions = actions_visitor.popActionsLevel();
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
/// There can not be aggregate functions inside the WHERE and PREWHERE.
|
||||
if (select_query && (ast.get() == select_query->where().get() || ast.get() == select_query->prewhere().get()))
|
||||
{
|
||||
assertNoAggregates(ast, "in WHERE or PREWHERE");
|
||||
return;
|
||||
}
|
||||
|
||||
/// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it.
|
||||
if (!select_query)
|
||||
{
|
||||
assertNoAggregates(ast, "in wrong place");
|
||||
return;
|
||||
}
|
||||
|
||||
const auto * node = ast->as<ASTFunction>();
|
||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
{
|
||||
has_aggregation = true;
|
||||
AggregateDescription aggregate;
|
||||
aggregate.column_name = node->getColumnName();
|
||||
|
||||
/// Make unique aggregate functions.
|
||||
for (size_t i = 0; i < aggregate_descriptions.size(); ++i)
|
||||
if (aggregate_descriptions[i].column_name == aggregate.column_name)
|
||||
return;
|
||||
|
||||
const ASTs & arguments = node->arguments->children;
|
||||
aggregate.argument_names.resize(arguments.size());
|
||||
DataTypes types(arguments.size());
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
{
|
||||
/// There can not be other aggregate functions within the aggregate functions.
|
||||
assertNoAggregates(arguments[i], "inside another aggregate function");
|
||||
|
||||
getRootActions(arguments[i], true, actions);
|
||||
const std::string & name = arguments[i]->getColumnName();
|
||||
types[i] = actions->getSampleBlock().getByName(name).type;
|
||||
@ -406,41 +348,24 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr
|
||||
|
||||
aggregate_descriptions.push_back(aggregate);
|
||||
}
|
||||
else
|
||||
{
|
||||
for (const auto & child : ast->children)
|
||||
if (!child->as<ASTSubquery>() && !child->as<ASTSelectQuery>())
|
||||
getAggregates(child, actions);
|
||||
}
|
||||
|
||||
return !aggregates().empty();
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::assertNoAggregates(const ASTPtr & ast, const char * description)
|
||||
{
|
||||
const auto * node = ast->as<ASTFunction>();
|
||||
|
||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
throw Exception("Aggregate function " + node->getColumnName()
|
||||
+ " is found " + String(description) + " in query", ErrorCodes::ILLEGAL_AGGREGATION);
|
||||
|
||||
for (const auto & child : ast->children)
|
||||
if (!child->as<ASTSubquery>() && !child->as<ASTSelectQuery>())
|
||||
assertNoAggregates(child, description);
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::assertSelect() const
|
||||
const ASTSelectQuery * ExpressionAnalyzer::getSelectQuery() const
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
if (!select_query)
|
||||
throw Exception("Not a select query", ErrorCodes::LOGICAL_ERROR);
|
||||
return select_query;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::assertAggregation() const
|
||||
const ASTSelectQuery * SelectQueryExpressionAnalyzer::getAggregatingQuery() const
|
||||
{
|
||||
if (!has_aggregation)
|
||||
throw Exception("No aggregation", ErrorCodes::LOGICAL_ERROR);
|
||||
return getSelectQuery();
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const
|
||||
@ -468,11 +393,9 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio
|
||||
actions->add(ExpressionAction::arrayJoin(result_columns, array_join_is_left, context));
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertSelect();
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
bool is_array_join_left;
|
||||
ASTPtr array_join_expression_list = select_query->array_join_expression_list(is_array_join_left);
|
||||
@ -489,17 +412,6 @@ bool ExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & chain, bool on
|
||||
return true;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, bool only_types) const
|
||||
{
|
||||
if (only_types)
|
||||
actions->add(ExpressionAction::ordinaryJoin(nullptr, analyzedJoin().key_names_left, columnsAddedByJoin()));
|
||||
else
|
||||
for (auto & subquery_for_set : subqueries_for_sets)
|
||||
if (subquery_for_set.second.join)
|
||||
actions->add(ExpressionAction::ordinaryJoin(subquery_for_set.second.join, analyzedJoin().key_names_left,
|
||||
columnsAddedByJoin()));
|
||||
}
|
||||
|
||||
static void appendRequiredColumns(
|
||||
NameSet & required_columns, const Block & sample, const Names & key_names_right, const NamesAndTypesList & columns_added_by_join)
|
||||
{
|
||||
@ -512,41 +424,37 @@ static void appendRequiredColumns(
|
||||
required_columns.insert(column.name);
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
/// It's possible to set nullptr as join for only_types mode
|
||||
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, JoinPtr join) const
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
actions->add(ExpressionAction::ordinaryJoin(join, analyzedJoin().key_names_left, columnsAddedByJoin()));
|
||||
}
|
||||
|
||||
assertSelect();
|
||||
|
||||
if (!select_query->join())
|
||||
bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const ASTTablesInSelectQueryElement * ast_join = getSelectQuery()->join();
|
||||
if (!ast_join)
|
||||
return false;
|
||||
|
||||
SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join);
|
||||
|
||||
ASTPtr left_keys_list = std::make_shared<ASTExpressionList>();
|
||||
left_keys_list->children = analyzedJoin().key_asts_left;
|
||||
|
||||
initChain(chain, sourceColumns());
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
|
||||
const auto & join_element = select_query->join()->as<ASTTablesInSelectQueryElement &>();
|
||||
getRootActions(left_keys_list, only_types, step.actions);
|
||||
addJoinAction(step.actions, subquery_for_set.join);
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
static JoinPtr tryGetStorageJoin(const ASTTablesInSelectQueryElement & join_element, const Context & context)
|
||||
{
|
||||
const auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>();
|
||||
auto & join_params = join_element.table_join->as<ASTTableJoin &>();
|
||||
|
||||
if (join_params.strictness == ASTTableJoin::Strictness::Unspecified && join_params.kind != ASTTableJoin::Kind::Cross)
|
||||
{
|
||||
if (settings.join_default_strictness == "ANY")
|
||||
join_params.strictness = ASTTableJoin::Strictness::Any;
|
||||
else if (settings.join_default_strictness == "ALL")
|
||||
join_params.strictness = ASTTableJoin::Strictness::All;
|
||||
else
|
||||
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
|
||||
}
|
||||
|
||||
const auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>();
|
||||
|
||||
getActionsFromJoinKeys(join_params, only_types, step.actions);
|
||||
|
||||
/// Two JOINs are not supported with the same subquery, but different USINGs.
|
||||
auto join_hash = join_element.getTreeHash();
|
||||
|
||||
SubqueryForSet & subquery_for_set = subqueries_for_sets[toString(join_hash.first) + "_" + toString(join_hash.second)];
|
||||
|
||||
/// Special case - if table name is specified on the right of JOIN, then the table has the type Join (the previously prepared mapping).
|
||||
/// TODO This syntax does not support specifying a database name.
|
||||
if (table_to_join.database_and_table_name)
|
||||
{
|
||||
@ -562,72 +470,106 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
storage_join->assertCompatible(join_params.kind, join_params.strictness);
|
||||
/// TODO Check the set of keys.
|
||||
|
||||
JoinPtr & join = storage_join->getJoin();
|
||||
subquery_for_set.join = join;
|
||||
return storage_join->getJoin();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!subquery_for_set.join)
|
||||
{
|
||||
auto & analyzed_join = analyzedJoin();
|
||||
/// Actions which need to be calculated on joined block.
|
||||
ExpressionActionsPtr joined_block_actions =
|
||||
analyzed_join.createJoinedBlockActions(columnsAddedByJoin(), select_query, context);
|
||||
|
||||
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
|
||||
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
|
||||
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
|
||||
* - this function shows the expression JOIN _data1.
|
||||
*/
|
||||
if (!subquery_for_set.source)
|
||||
{
|
||||
ASTPtr table;
|
||||
|
||||
if (table_to_join.subquery)
|
||||
table = table_to_join.subquery;
|
||||
else if (table_to_join.table_function)
|
||||
table = table_to_join.table_function;
|
||||
else if (table_to_join.database_and_table_name)
|
||||
table = table_to_join.database_and_table_name;
|
||||
|
||||
Names action_columns = joined_block_actions->getRequiredColumns();
|
||||
NameSet required_columns(action_columns.begin(), action_columns.end());
|
||||
|
||||
appendRequiredColumns(
|
||||
required_columns, joined_block_actions->getSampleBlock(), analyzed_join.key_names_right, columnsAddedByJoin());
|
||||
|
||||
auto original_map = analyzed_join.getOriginalColumnsMap(required_columns);
|
||||
Names original_columns;
|
||||
for (auto & pr : original_map)
|
||||
original_columns.push_back(pr.second);
|
||||
|
||||
auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns);
|
||||
|
||||
subquery_for_set.makeSource(interpreter, original_map);
|
||||
}
|
||||
|
||||
Block sample_block = subquery_for_set.renamedSampleBlock();
|
||||
joined_block_actions->execute(sample_block);
|
||||
|
||||
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
|
||||
subquery_for_set.join = std::make_shared<Join>(analyzedJoin().key_names_right, settings.join_use_nulls,
|
||||
settings.size_limits_for_join, join_params.kind, join_params.strictness);
|
||||
subquery_for_set.join->setSampleBlock(sample_block);
|
||||
subquery_for_set.joined_block_actions = joined_block_actions;
|
||||
}
|
||||
|
||||
addJoinAction(step.actions, false);
|
||||
|
||||
return true;
|
||||
return {};
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendPrewhere(
|
||||
SubqueryForSet & SelectQueryExpressionAnalyzer::getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element)
|
||||
{
|
||||
/// Two JOINs are not supported with the same subquery, but different USINGs.
|
||||
auto join_hash = join_element.getTreeHash();
|
||||
String join_subquery_id = toString(join_hash.first) + "_" + toString(join_hash.second);
|
||||
|
||||
SubqueryForSet & subquery_for_set = subqueries_for_sets[join_subquery_id];
|
||||
|
||||
/// Special case - if table name is specified on the right of JOIN, then the table has the type Join (the previously prepared mapping).
|
||||
if (!subquery_for_set.join)
|
||||
subquery_for_set.join = tryGetStorageJoin(join_element, context);
|
||||
|
||||
if (!subquery_for_set.join)
|
||||
makeHashJoin(join_element, subquery_for_set);
|
||||
|
||||
return subquery_for_set;
|
||||
}
|
||||
|
||||
void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryElement & join_element,
|
||||
SubqueryForSet & subquery_for_set) const
|
||||
{
|
||||
/// Actions which need to be calculated on joined block.
|
||||
ExpressionActionsPtr joined_block_actions = createJoinedBlockActions();
|
||||
|
||||
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
|
||||
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
|
||||
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
|
||||
* - this function shows the expression JOIN _data1.
|
||||
*/
|
||||
if (!subquery_for_set.source)
|
||||
{
|
||||
ASTPtr table;
|
||||
|
||||
auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>();
|
||||
if (table_to_join.subquery)
|
||||
table = table_to_join.subquery;
|
||||
else if (table_to_join.table_function)
|
||||
table = table_to_join.table_function;
|
||||
else if (table_to_join.database_and_table_name)
|
||||
table = table_to_join.database_and_table_name;
|
||||
|
||||
Names action_columns = joined_block_actions->getRequiredColumns();
|
||||
NameSet required_columns(action_columns.begin(), action_columns.end());
|
||||
|
||||
auto & analyzed_join = analyzedJoin();
|
||||
appendRequiredColumns(
|
||||
required_columns, joined_block_actions->getSampleBlock(), analyzed_join.key_names_right, columnsAddedByJoin());
|
||||
|
||||
auto original_map = analyzed_join.getOriginalColumnsMap(required_columns);
|
||||
Names original_columns;
|
||||
for (auto & pr : original_map)
|
||||
original_columns.push_back(pr.second);
|
||||
|
||||
auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns);
|
||||
|
||||
subquery_for_set.makeSource(interpreter, original_map);
|
||||
}
|
||||
|
||||
Block sample_block = subquery_for_set.renamedSampleBlock();
|
||||
joined_block_actions->execute(sample_block);
|
||||
|
||||
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
|
||||
auto & join_params = join_element.table_join->as<ASTTableJoin &>();
|
||||
subquery_for_set.join = std::make_shared<Join>(analyzedJoin().key_names_right, settings.join_use_nulls,
|
||||
settings.size_limits_for_join, join_params.kind, join_params.strictness);
|
||||
subquery_for_set.join->setSampleBlock(sample_block);
|
||||
subquery_for_set.joined_block_actions = joined_block_actions;
|
||||
}
|
||||
|
||||
ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const
|
||||
{
|
||||
/// Create custom expression list with join keys from right table.
|
||||
ASTPtr expression_list = std::make_shared<ASTExpressionList>();
|
||||
ASTs & children = expression_list->children;
|
||||
|
||||
if (analyzedJoin().hasOn())
|
||||
for (const auto & join_right_key : analyzedJoin().key_asts_right)
|
||||
children.emplace_back(join_right_key);
|
||||
|
||||
NameSet required_columns_set(analyzedJoin().key_names_right.begin(), analyzedJoin().key_names_right.end());
|
||||
for (const auto & joined_column : columnsAddedByJoin())
|
||||
required_columns_set.insert(joined_column.name);
|
||||
Names required_columns(required_columns_set.begin(), required_columns_set.end());
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columns_from_joined_table, required_columns);
|
||||
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
|
||||
}
|
||||
|
||||
bool SelectQueryExpressionAnalyzer::appendPrewhere(
|
||||
ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertSelect();
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
if (!select_query->prewhere())
|
||||
return false;
|
||||
@ -698,11 +640,9 @@ bool ExpressionAnalyzer::appendPrewhere(
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertSelect();
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
if (!select_query->where())
|
||||
return false;
|
||||
@ -718,11 +658,9 @@ bool ExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, bool only_t
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertAggregation();
|
||||
const auto * select_query = getAggregatingQuery();
|
||||
|
||||
if (!select_query->groupBy())
|
||||
return false;
|
||||
@ -740,11 +678,9 @@ bool ExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only
|
||||
return true;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
|
||||
void SelectQueryExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertAggregation();
|
||||
const auto * select_query = getAggregatingQuery();
|
||||
|
||||
initChain(chain, sourceColumns());
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
@ -757,20 +693,27 @@ void ExpressionAnalyzer::appendAggregateFunctionsArguments(ExpressionActionsChai
|
||||
}
|
||||
}
|
||||
|
||||
getActionsBeforeAggregation(select_query->select(), step.actions, only_types);
|
||||
/// Collect aggregates removing duplicates by node.getColumnName()
|
||||
/// It's not clear why we recollect aggregates (for query parts) while we're able to use previously collected ones (for entire query)
|
||||
/// @note The original recollection logic didn't remove duplicates.
|
||||
GetAggregatesVisitor::Data data;
|
||||
GetAggregatesVisitor(data).visit(select_query->select());
|
||||
|
||||
if (select_query->having())
|
||||
getActionsBeforeAggregation(select_query->having(), step.actions, only_types);
|
||||
GetAggregatesVisitor(data).visit(select_query->having());
|
||||
|
||||
if (select_query->orderBy())
|
||||
getActionsBeforeAggregation(select_query->orderBy(), step.actions, only_types);
|
||||
GetAggregatesVisitor(data).visit(select_query->orderBy());
|
||||
|
||||
/// TODO: data.aggregates -> aggregates()
|
||||
for (const ASTFunction * node : data.aggregates)
|
||||
for (auto & argument : node->arguments->children)
|
||||
getRootActions(argument, only_types, step.actions);
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertAggregation();
|
||||
const auto * select_query = getAggregatingQuery();
|
||||
|
||||
if (!select_query->having())
|
||||
return false;
|
||||
@ -784,11 +727,9 @@ bool ExpressionAnalyzer::appendHaving(ExpressionActionsChain & chain, bool only_
|
||||
return true;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
|
||||
void SelectQueryExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertSelect();
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
initChain(chain, aggregated_columns);
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
@ -799,11 +740,9 @@ void ExpressionAnalyzer::appendSelect(ExpressionActionsChain & chain, bool only_
|
||||
step.required_output.push_back(child->getColumnName());
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertSelect();
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
if (!select_query->orderBy())
|
||||
return false;
|
||||
@ -825,11 +764,9 @@ bool ExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain, bool only
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
|
||||
bool SelectQueryExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only_types)
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertSelect();
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
if (!select_query->limitBy())
|
||||
return false;
|
||||
@ -856,11 +793,9 @@ bool ExpressionAnalyzer::appendLimitBy(ExpressionActionsChain & chain, bool only
|
||||
return true;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
|
||||
void SelectQueryExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) const
|
||||
{
|
||||
const auto * select_query = query->as<ASTSelectQuery>();
|
||||
|
||||
assertSelect();
|
||||
const auto * select_query = getSelectQuery();
|
||||
|
||||
initChain(chain, aggregated_columns);
|
||||
ExpressionActionsChain::Step & step = chain.steps.back();
|
||||
@ -891,19 +826,6 @@ void ExpressionAnalyzer::appendExpression(ExpressionActionsChain & chain, const
|
||||
}
|
||||
|
||||
|
||||
void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries)
|
||||
{
|
||||
const auto * node = ast->as<ASTFunction>();
|
||||
|
||||
if (node && AggregateFunctionFactory::instance().isAggregateFunctionName(node->name))
|
||||
for (auto & argument : node->arguments->children)
|
||||
getRootActions(argument, no_subqueries, actions);
|
||||
else
|
||||
for (auto & child : ast->children)
|
||||
getActionsBeforeAggregation(child, actions, no_subqueries);
|
||||
}
|
||||
|
||||
|
||||
ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result)
|
||||
{
|
||||
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(sourceColumns(), context);
|
||||
@ -959,7 +881,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getConstActions()
|
||||
return actions;
|
||||
}
|
||||
|
||||
void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const
|
||||
void SelectQueryExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const
|
||||
{
|
||||
for (const auto & name_and_type : aggregation_keys)
|
||||
key_names.emplace_back(name_and_type.name);
|
||||
|
@ -2,9 +2,9 @@
|
||||
|
||||
#include <Core/Settings.h>
|
||||
#include <DataStreams/IBlockStream_fwd.h>
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/SubqueryForSet.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
@ -24,6 +24,7 @@ struct ASTTableJoin;
|
||||
class ASTFunction;
|
||||
class ASTExpressionList;
|
||||
class ASTSelectQuery;
|
||||
struct ASTTablesInSelectQueryElement;
|
||||
|
||||
struct SyntaxAnalyzerResult;
|
||||
using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
|
||||
@ -31,9 +32,6 @@ using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
|
||||
/// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately.
|
||||
struct ExpressionAnalyzerData
|
||||
{
|
||||
/// If non-empty, ignore all expressions in not from this list.
|
||||
NameSet required_result_columns;
|
||||
|
||||
SubqueriesForSets subqueries_for_sets;
|
||||
PreparedSets prepared_sets;
|
||||
|
||||
@ -49,13 +47,6 @@ struct ExpressionAnalyzerData
|
||||
|
||||
/// All new temporary tables obtained by performing the GLOBAL IN/JOIN subqueries.
|
||||
Tables external_tables;
|
||||
|
||||
protected:
|
||||
ExpressionAnalyzerData(const NameSet & required_result_columns_,
|
||||
const SubqueriesForSets & subqueries_for_sets_)
|
||||
: required_result_columns(required_result_columns_),
|
||||
subqueries_for_sets(subqueries_for_sets_)
|
||||
{}
|
||||
};
|
||||
|
||||
|
||||
@ -63,7 +54,7 @@ protected:
|
||||
*
|
||||
* NOTE: if `ast` is a SELECT query from a table, the structure of this table should not change during the lifetime of ExpressionAnalyzer.
|
||||
*/
|
||||
class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncopyable
|
||||
class ExpressionAnalyzer : protected ExpressionAnalyzerData, private boost::noncopyable
|
||||
{
|
||||
private:
|
||||
/// Extracts settings to enlight which are used (and avoid copy of others).
|
||||
@ -73,33 +64,120 @@ private:
|
||||
const bool join_use_nulls;
|
||||
const SizeLimits size_limits_for_set;
|
||||
const SizeLimits size_limits_for_join;
|
||||
const String join_default_strictness;
|
||||
|
||||
ExtractedSettings(const Settings & settings_)
|
||||
: use_index_for_in_with_subqueries(settings_.use_index_for_in_with_subqueries),
|
||||
join_use_nulls(settings_.join_use_nulls),
|
||||
size_limits_for_set(settings_.max_rows_in_set, settings_.max_bytes_in_set, settings_.set_overflow_mode),
|
||||
size_limits_for_join(settings_.max_rows_in_join, settings_.max_bytes_in_join, settings_.join_overflow_mode),
|
||||
join_default_strictness(settings_.join_default_strictness.toString())
|
||||
size_limits_for_join(settings_.max_rows_in_join, settings_.max_bytes_in_join, settings_.join_overflow_mode)
|
||||
{}
|
||||
};
|
||||
|
||||
public:
|
||||
/// Ctor for non-select queries. Generally its usage is:
|
||||
/// auto actions = ExpressionAnalyzer(query, syntax, context).getActions();
|
||||
ExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_)
|
||||
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, 0, false)
|
||||
{}
|
||||
|
||||
void appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types);
|
||||
|
||||
/// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression.
|
||||
/// If add_aliases, only the calculated values in the desired order and add aliases.
|
||||
/// If also project_result, than only aliases remain in the output block.
|
||||
/// Otherwise, only temporary columns will be deleted from the block.
|
||||
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true);
|
||||
|
||||
/// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants.
|
||||
/// Does not execute subqueries.
|
||||
ExpressionActionsPtr getConstActions();
|
||||
|
||||
/** Sets that require a subquery to be create.
|
||||
* Only the sets needed to perform actions returned from already executed `append*` or `getActions`.
|
||||
* That is, you need to call getSetsWithSubqueries after all calls of `append*` or `getActions`
|
||||
* and create all the returned sets before performing the actions.
|
||||
*/
|
||||
const SubqueriesForSets & getSubqueriesForSets() const { return subqueries_for_sets; }
|
||||
|
||||
/// Get intermediates for tests
|
||||
const ExpressionAnalyzerData & getAnalyzedData() const { return *this; }
|
||||
|
||||
protected:
|
||||
ExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
size_t subquery_depth_,
|
||||
bool do_global_);
|
||||
|
||||
ASTPtr query;
|
||||
const Context & context;
|
||||
const ExtractedSettings settings;
|
||||
size_t subquery_depth;
|
||||
|
||||
SyntaxAnalyzerResultPtr syntax;
|
||||
|
||||
const StoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
|
||||
const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; }
|
||||
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
|
||||
const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; }
|
||||
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
|
||||
|
||||
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
||||
void initGlobalSubqueriesAndExternalTables(bool do_global);
|
||||
|
||||
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
|
||||
|
||||
void addJoinAction(ExpressionActionsPtr & actions, JoinPtr join = {}) const;
|
||||
|
||||
void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
|
||||
|
||||
/** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions,
|
||||
* Create a set of columns aggregated_columns resulting after the aggregation, if any,
|
||||
* or after all the actions that are normally performed before aggregation.
|
||||
* Set has_aggregation = true if there is GROUP BY or at least one aggregate function.
|
||||
*/
|
||||
void analyzeAggregation();
|
||||
bool makeAggregateDescriptions(ExpressionActionsPtr & actions);
|
||||
|
||||
/// columns - the columns that are present before the transformations begin.
|
||||
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
|
||||
|
||||
const ASTSelectQuery * getSelectQuery() const;
|
||||
|
||||
bool isRemoteStorage() const;
|
||||
};
|
||||
|
||||
/// SelectQuery specific ExpressionAnalyzer part.
|
||||
class SelectQueryExpressionAnalyzer : public ExpressionAnalyzer
|
||||
{
|
||||
public:
|
||||
SelectQueryExpressionAnalyzer(
|
||||
const ASTPtr & query_,
|
||||
const SyntaxAnalyzerResultPtr & syntax_analyzer_result_,
|
||||
const Context & context_,
|
||||
const NameSet & required_result_columns_ = {},
|
||||
size_t subquery_depth_ = 0,
|
||||
bool do_global_ = false,
|
||||
const SubqueriesForSets & subqueries_for_set_ = {});
|
||||
bool do_global_ = false)
|
||||
: ExpressionAnalyzer(query_, syntax_analyzer_result_, context_, subquery_depth_, do_global_)
|
||||
, required_result_columns(required_result_columns_)
|
||||
{}
|
||||
|
||||
/// Does the expression have aggregate functions or a GROUP BY or HAVING section.
|
||||
bool hasAggregation() const { return has_aggregation; }
|
||||
bool hasGlobalSubqueries() { return has_global_subqueries; }
|
||||
|
||||
/// Get a list of aggregation keys and descriptions of aggregate functions if the query contains GROUP BY.
|
||||
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const;
|
||||
|
||||
const PreparedSets & getPreparedSets() const { return prepared_sets; }
|
||||
|
||||
/// Tables that will need to be sent to remote servers for distributed query processing.
|
||||
const Tables & getExternalTables() const { return external_tables; }
|
||||
|
||||
/** These methods allow you to build a chain of transformations over a block, that receives values in the desired sections of the query.
|
||||
*
|
||||
* Example usage:
|
||||
@ -132,81 +210,12 @@ public:
|
||||
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
|
||||
void appendProjectResult(ExpressionActionsChain & chain) const;
|
||||
|
||||
void appendExpression(ExpressionActionsChain & chain, const ASTPtr & expr, bool only_types);
|
||||
|
||||
/// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression.
|
||||
/// If add_aliases, only the calculated values in the desired order and add aliases.
|
||||
/// If also project_result, than only aliases remain in the output block.
|
||||
/// Otherwise, only temporary columns will be deleted from the block.
|
||||
ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true);
|
||||
|
||||
/// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants.
|
||||
/// Does not execute subqueries.
|
||||
ExpressionActionsPtr getConstActions();
|
||||
|
||||
/** Sets that require a subquery to be create.
|
||||
* Only the sets needed to perform actions returned from already executed `append*` or `getActions`.
|
||||
* That is, you need to call getSetsWithSubqueries after all calls of `append*` or `getActions`
|
||||
* and create all the returned sets before performing the actions.
|
||||
*/
|
||||
const SubqueriesForSets & getSubqueriesForSets() const { return subqueries_for_sets; }
|
||||
|
||||
const PreparedSets & getPreparedSets() const { return prepared_sets; }
|
||||
|
||||
/** Tables that will need to be sent to remote servers for distributed query processing.
|
||||
*/
|
||||
const Tables & getExternalTables() const { return external_tables; }
|
||||
|
||||
/// Get intermediates for tests
|
||||
const ExpressionAnalyzerData & getAnalyzedData() const { return *this; }
|
||||
|
||||
/// Create Set-s that we can from IN section to use the index on them.
|
||||
void makeSetsForIndex();
|
||||
|
||||
bool hasGlobalSubqueries() { return has_global_subqueries; }
|
||||
void makeSetsForIndex(const ASTPtr & node);
|
||||
|
||||
private:
|
||||
ASTPtr query;
|
||||
const Context & context;
|
||||
const ExtractedSettings settings;
|
||||
size_t subquery_depth;
|
||||
bool do_global; /// Do I need to prepare for execution global subqueries when analyzing the query.
|
||||
|
||||
SyntaxAnalyzerResultPtr syntax;
|
||||
|
||||
const StoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
|
||||
const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; }
|
||||
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
|
||||
const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; }
|
||||
|
||||
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
|
||||
void initGlobalSubqueriesAndExternalTables();
|
||||
|
||||
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
|
||||
|
||||
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
|
||||
|
||||
/// If ast is ASTSelectQuery with JOIN, add actions for JOIN key columns.
|
||||
void getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions);
|
||||
|
||||
void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);
|
||||
|
||||
void getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries);
|
||||
|
||||
/** Add aggregation keys to aggregation_keys, aggregate functions to aggregate_descriptions,
|
||||
* Create a set of columns aggregated_columns resulting after the aggregation, if any,
|
||||
* or after all the actions that are normally performed before aggregation.
|
||||
* Set has_aggregation = true if there is GROUP BY or at least one aggregate function.
|
||||
*/
|
||||
void analyzeAggregation();
|
||||
void getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions);
|
||||
void assertNoAggregates(const ASTPtr & ast, const char * description);
|
||||
|
||||
/// columns - the columns that are present before the transformations begin.
|
||||
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
|
||||
|
||||
void assertSelect() const;
|
||||
void assertAggregation() const;
|
||||
/// If non-empty, ignore all expressions not from this list.
|
||||
NameSet required_result_columns;
|
||||
|
||||
/**
|
||||
* Create Set from a subquery or a table expression in the query. The created set is suitable for using the index.
|
||||
@ -214,9 +223,11 @@ private:
|
||||
*/
|
||||
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name);
|
||||
|
||||
void makeSetsForIndexImpl(const ASTPtr & node);
|
||||
SubqueryForSet & getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element);
|
||||
ExpressionActionsPtr createJoinedBlockActions() const;
|
||||
void makeHashJoin(const ASTTablesInSelectQueryElement & join_element, SubqueryForSet & subquery_for_set) const;
|
||||
|
||||
bool isRemoteStorage() const;
|
||||
const ASTSelectQuery * getAggregatingQuery() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
75
dbms/src/Interpreters/GetAggregatesVisitor.h
Normal file
75
dbms/src/Interpreters/GetAggregatesVisitor.h
Normal file
@ -0,0 +1,75 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_AGGREGATION;
|
||||
}
|
||||
|
||||
class GetAggregatesMatcher
|
||||
{
|
||||
public:
|
||||
using Visitor = ConstInDepthNodeVisitor<GetAggregatesMatcher, true>;
|
||||
|
||||
struct Data
|
||||
{
|
||||
const char * assert_no_aggregates = nullptr;
|
||||
std::unordered_set<String> uniq_names;
|
||||
std::vector<const ASTFunction *> aggregates;
|
||||
};
|
||||
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child)
|
||||
{
|
||||
if (child->as<ASTSubquery>() || child->as<ASTSelectQuery>())
|
||||
return false;
|
||||
if (auto * func = node->as<ASTFunction>())
|
||||
if (isAggregateFunction(func->name))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
|
||||
static void visit(const ASTPtr & ast, Data & data)
|
||||
{
|
||||
if (auto * func = ast->as<ASTFunction>())
|
||||
visit(*func, ast, data);
|
||||
}
|
||||
|
||||
private:
|
||||
static void visit(const ASTFunction & node, const ASTPtr &, Data & data)
|
||||
{
|
||||
if (!isAggregateFunction(node.name))
|
||||
return;
|
||||
|
||||
if (data.assert_no_aggregates)
|
||||
throw Exception("Aggregate function " + node.getColumnName() + " is found " + String(data.assert_no_aggregates) + " in query",
|
||||
ErrorCodes::ILLEGAL_AGGREGATION);
|
||||
|
||||
String column_name = node.getColumnName();
|
||||
if (data.uniq_names.count(column_name))
|
||||
return;
|
||||
|
||||
data.uniq_names.insert(column_name);
|
||||
data.aggregates.push_back(&node);
|
||||
}
|
||||
|
||||
static bool isAggregateFunction(const String & name)
|
||||
{
|
||||
return AggregateFunctionFactory::instance().isAggregateFunctionName(name);
|
||||
}
|
||||
};
|
||||
|
||||
using GetAggregatesVisitor = GetAggregatesMatcher::Visitor;
|
||||
|
||||
|
||||
inline void assertNoAggregates(const ASTPtr & ast, const char * description)
|
||||
{
|
||||
GetAggregatesVisitor::Data data{description, {}, {}};
|
||||
GetAggregatesVisitor(data).visit(ast);
|
||||
}
|
||||
|
||||
}
|
@ -7,7 +7,6 @@
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/ActionsVisitor.h>
|
||||
#include <Interpreters/interpretSubquery.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/Block.h>
|
||||
|
@ -144,7 +144,7 @@ String generateFilterActions(ExpressionActionsPtr & actions, const StoragePtr &
|
||||
|
||||
/// Using separate expression analyzer to prevent any possible alias injection
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query_ast, storage->getColumns().getAllPhysical());
|
||||
ExpressionAnalyzer analyzer(query_ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(query_ast, syntax_result, context);
|
||||
ExpressionActionsChain new_chain(context);
|
||||
analyzer.appendSelect(new_chain, false);
|
||||
actions = new_chain.getLastActions();
|
||||
@ -296,7 +296,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze(
|
||||
query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList());
|
||||
query_analyzer = std::make_unique<ExpressionAnalyzer>(
|
||||
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
|
||||
query_ptr, syntax_analyzer_result, context,
|
||||
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
|
||||
options.subquery_depth, !options.only_analyze);
|
||||
@ -665,7 +665,8 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c
|
||||
}
|
||||
|
||||
|
||||
static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, const ASTSelectQuery & query, const Context & context)
|
||||
static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, const ASTSelectQuery & query,
|
||||
const Context & context, const SyntaxAnalyzerResultPtr & global_syntax_result)
|
||||
{
|
||||
if (!merge_tree.hasSortingKey())
|
||||
return {};
|
||||
@ -677,38 +678,21 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons
|
||||
const auto & sorting_key_columns = merge_tree.getSortingKeyColumns();
|
||||
size_t prefix_size = std::min(order_descr.size(), sorting_key_columns.size());
|
||||
|
||||
auto order_by_expr = query.orderBy();
|
||||
SyntaxAnalyzerResultPtr syntax_result;
|
||||
try
|
||||
{
|
||||
syntax_result = SyntaxAnalyzer(context).analyze(order_by_expr, merge_tree.getColumns().getAllPhysical());
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
return {};
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < prefix_size; ++i)
|
||||
{
|
||||
/// Read in pk order in case of exact match with order key element
|
||||
if (global_syntax_result->array_join_result_to_source.count(order_descr[i].column_name))
|
||||
break;
|
||||
|
||||
/// Optimize in case of exact match with order key element
|
||||
/// or in some simple cases when order key element is wrapped into monotonic function.
|
||||
int current_direction = order_descr[i].direction;
|
||||
if (order_descr[i].column_name == sorting_key_columns[i] && current_direction == read_direction)
|
||||
prefix_order_descr.push_back(order_descr[i]);
|
||||
else
|
||||
{
|
||||
const auto & ast = query.orderBy()->children[i];
|
||||
ExpressionActionsPtr actions;
|
||||
try
|
||||
{
|
||||
actions = ExpressionAnalyzer(ast->children.at(0), syntax_result, context).getActions(false);
|
||||
}
|
||||
catch (const Exception &)
|
||||
{
|
||||
/// Can't analyze order expression at this stage.
|
||||
/// May be some actions required for order will be executed later.
|
||||
break;
|
||||
}
|
||||
auto ast = query.orderBy()->children[i]->children.at(0);
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(ast, global_syntax_result->required_source_columns);
|
||||
auto actions = ExpressionAnalyzer(ast, syntax_result, context).getActions(true);
|
||||
|
||||
const auto & input_columns = actions->getRequiredColumnsWithTypes();
|
||||
if (input_columns.size() != 1 || input_columns.front().name != sorting_key_columns[i])
|
||||
@ -790,7 +774,8 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
|
||||
if (!dry_run)
|
||||
from_stage = storage->getQueryProcessingStage(context);
|
||||
|
||||
query_analyzer->makeSetsForIndex();
|
||||
query_analyzer->makeSetsForIndex(query.where());
|
||||
query_analyzer->makeSetsForIndex(query.prewhere());
|
||||
|
||||
auto optimize_prewhere = [&](auto & merge_tree)
|
||||
{
|
||||
@ -820,10 +805,10 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
|
||||
}
|
||||
|
||||
SortingInfoPtr sorting_info;
|
||||
if (settings.optimize_read_in_order && storage && query.orderBy() && !query.groupBy() && !query.final())
|
||||
if (settings.optimize_read_in_order && storage && query.orderBy() && !query.groupBy() && !query.final() && !query.join())
|
||||
{
|
||||
if (const MergeTreeData * merge_tree_data = dynamic_cast<const MergeTreeData *>(storage.get()))
|
||||
sorting_info = optimizeReadInOrder(*merge_tree_data, query, context);
|
||||
sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result);
|
||||
}
|
||||
|
||||
if (dry_run)
|
||||
@ -1941,13 +1926,12 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so
|
||||
* At this stage we merge per-thread streams into one.
|
||||
*/
|
||||
|
||||
if (sorting_info->prefix_order_descr.size() < order_descr.size())
|
||||
bool need_finish_sorting = (sorting_info->prefix_order_descr.size() < order_descr.size());
|
||||
if (need_finish_sorting)
|
||||
{
|
||||
pipeline.transform([&](auto & stream)
|
||||
{
|
||||
stream = std::make_shared<FinishSortingBlockInputStream>(
|
||||
stream, sorting_info->prefix_order_descr,
|
||||
order_descr, settings.max_block_size, limit);
|
||||
stream = std::make_shared<PartialSortingBlockInputStream>(stream, order_descr, limit);
|
||||
});
|
||||
}
|
||||
|
||||
@ -1958,11 +1942,19 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so
|
||||
stream = std::make_shared<AsynchronousBlockInputStream>(stream);
|
||||
});
|
||||
|
||||
UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit);
|
||||
pipeline.firstStream() = std::make_shared<MergingSortedBlockInputStream>(
|
||||
pipeline.streams, order_descr,
|
||||
settings.max_block_size, limit);
|
||||
pipeline.streams, sorting_info->prefix_order_descr,
|
||||
settings.max_block_size, limit_for_merging);
|
||||
pipeline.streams.resize(1);
|
||||
}
|
||||
|
||||
if (need_finish_sorting)
|
||||
{
|
||||
pipeline.firstStream() = std::make_shared<FinishSortingBlockInputStream>(
|
||||
pipeline.firstStream(), sorting_info->prefix_order_descr,
|
||||
order_descr, settings.max_block_size, limit);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
|
@ -248,7 +248,7 @@ private:
|
||||
Context context;
|
||||
NamesAndTypesList source_columns;
|
||||
SyntaxAnalyzerResultPtr syntax_analyzer_result;
|
||||
std::unique_ptr<ExpressionAnalyzer> query_analyzer;
|
||||
std::unique_ptr<SelectQueryExpressionAnalyzer> query_analyzer;
|
||||
SelectQueryInfo query_info;
|
||||
|
||||
/// How many streams we ask for storage to produce, and in how many threads we will do further processing.
|
||||
|
@ -16,6 +16,7 @@
|
||||
#include <Interpreters/QueryThreadLog.h>
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/TextLog.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Storages/StorageDistributed.h>
|
||||
#include <Storages/StorageReplicatedMergeTree.h>
|
||||
@ -232,7 +233,8 @@ BlockIO InterpreterSystemQuery::execute()
|
||||
[&] () { if (auto part_log = context.getPartLog("")) part_log->flush(); },
|
||||
[&] () { if (auto query_thread_log = context.getQueryThreadLog()) query_thread_log->flush(); },
|
||||
[&] () { if (auto trace_log = context.getTraceLog()) trace_log->flush(); },
|
||||
[&] () { if (auto text_log = context.getTextLog()) text_log->flush(); }
|
||||
[&] () { if (auto text_log = context.getTextLog()) text_log->flush(); },
|
||||
[&] () { if (auto metric_log = context.getMetricLog()) metric_log->flush(); }
|
||||
);
|
||||
break;
|
||||
case Type::STOP_LISTEN_QUERIES:
|
||||
|
133
dbms/src/Interpreters/MetricLog.cpp
Normal file
133
dbms/src/Interpreters/MetricLog.cpp
Normal file
@ -0,0 +1,133 @@
|
||||
#include <Interpreters/MetricLog.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
Block MetricLogElement::createBlock()
|
||||
{
|
||||
ColumnsWithTypeAndName columns_with_type_and_name;
|
||||
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDate>(), "event_date");
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeDateTime>(), "event_time");
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), "milliseconds");
|
||||
|
||||
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
|
||||
{
|
||||
std::string name;
|
||||
name += "ProfileEvent_";
|
||||
name += ProfileEvents::getName(ProfileEvents::Event(i));
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeUInt64>(), std::move(name));
|
||||
}
|
||||
|
||||
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
|
||||
{
|
||||
std::string name;
|
||||
name += "CurrentMetric_";
|
||||
name += CurrentMetrics::getName(ProfileEvents::Event(i));
|
||||
columns_with_type_and_name.emplace_back(std::make_shared<DataTypeInt64>(), std::move(name));
|
||||
}
|
||||
|
||||
return Block(columns_with_type_and_name);
|
||||
}
|
||||
|
||||
|
||||
void MetricLogElement::appendToBlock(Block & block) const
|
||||
{
|
||||
MutableColumns columns = block.mutateColumns();
|
||||
|
||||
size_t column_idx = 0;
|
||||
|
||||
columns[column_idx++]->insert(DateLUT::instance().toDayNum(event_time));
|
||||
columns[column_idx++]->insert(event_time);
|
||||
columns[column_idx++]->insert(milliseconds);
|
||||
|
||||
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
|
||||
columns[column_idx++]->insert(profile_events[i]);
|
||||
|
||||
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
|
||||
columns[column_idx++]->insert(current_metrics[i]);
|
||||
}
|
||||
|
||||
|
||||
void MetricLog::startCollectMetric(size_t collect_interval_milliseconds_)
|
||||
{
|
||||
collect_interval_milliseconds = collect_interval_milliseconds_;
|
||||
is_shutdown_metric_thread = false;
|
||||
metric_flush_thread = ThreadFromGlobalPool([this] { metricThreadFunction(); });
|
||||
}
|
||||
|
||||
|
||||
void MetricLog::stopCollectMetric()
|
||||
{
|
||||
bool old_val = false;
|
||||
if (!is_shutdown_metric_thread.compare_exchange_strong(old_val, true))
|
||||
return;
|
||||
metric_flush_thread.join();
|
||||
}
|
||||
|
||||
|
||||
inline UInt64 time_in_milliseconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
|
||||
{
|
||||
return std::chrono::duration_cast<std::chrono::milliseconds>(timepoint.time_since_epoch()).count();
|
||||
}
|
||||
|
||||
|
||||
inline UInt64 time_in_seconds(std::chrono::time_point<std::chrono::system_clock> timepoint)
|
||||
{
|
||||
return std::chrono::duration_cast<std::chrono::seconds>(timepoint.time_since_epoch()).count();
|
||||
}
|
||||
|
||||
|
||||
void MetricLog::metricThreadFunction()
|
||||
{
|
||||
auto desired_timepoint = std::chrono::system_clock::now();
|
||||
|
||||
/// For differentiation of ProfileEvents counters.
|
||||
std::vector<ProfileEvents::Count> prev_profile_events(ProfileEvents::end());
|
||||
|
||||
while (!is_shutdown_metric_thread)
|
||||
{
|
||||
try
|
||||
{
|
||||
const auto current_time = std::chrono::system_clock::now();
|
||||
|
||||
MetricLogElement elem;
|
||||
elem.event_time = std::chrono::system_clock::to_time_t(current_time);
|
||||
elem.milliseconds = time_in_milliseconds(current_time) - time_in_seconds(current_time) * 1000;
|
||||
|
||||
elem.profile_events.resize(ProfileEvents::end());
|
||||
for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i)
|
||||
{
|
||||
const ProfileEvents::Count new_value = ProfileEvents::global_counters[i].load(std::memory_order_relaxed);
|
||||
UInt64 & old_value = prev_profile_events[i];
|
||||
elem.profile_events[i] = new_value - old_value;
|
||||
old_value = new_value;
|
||||
}
|
||||
|
||||
elem.current_metrics.resize(CurrentMetrics::end());
|
||||
for (size_t i = 0, end = CurrentMetrics::end(); i < end; ++i)
|
||||
{
|
||||
elem.current_metrics[i] = CurrentMetrics::values[i];
|
||||
}
|
||||
|
||||
this->add(elem);
|
||||
|
||||
/// We will record current time into table but align it to regular time intervals to avoid time drift.
|
||||
/// We may drop some time points if the server is overloaded and recording took too much time.
|
||||
while (desired_timepoint <= current_time)
|
||||
desired_timepoint += std::chrono::milliseconds(collect_interval_milliseconds);
|
||||
|
||||
std::this_thread::sleep_until(desired_timepoint);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
tryLogCurrentException(__PRETTY_FUNCTION__);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
51
dbms/src/Interpreters/MetricLog.h
Normal file
51
dbms/src/Interpreters/MetricLog.h
Normal file
@ -0,0 +1,51 @@
|
||||
#pragma once
|
||||
|
||||
#include <Interpreters/SystemLog.h>
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
||||
#include <vector>
|
||||
#include <atomic>
|
||||
#include <ctime>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** MetricLog is a log of metric values measured at regular time interval.
|
||||
*/
|
||||
|
||||
struct MetricLogElement
|
||||
{
|
||||
time_t event_time{};
|
||||
UInt64 milliseconds{};
|
||||
|
||||
std::vector<ProfileEvents::Count> profile_events;
|
||||
std::vector<CurrentMetrics::Metric> current_metrics;
|
||||
|
||||
static std::string name() { return "MetricLog"; }
|
||||
static Block createBlock();
|
||||
void appendToBlock(Block & block) const;
|
||||
};
|
||||
|
||||
|
||||
class MetricLog : public SystemLog<MetricLogElement>
|
||||
{
|
||||
using SystemLog<MetricLogElement>::SystemLog;
|
||||
|
||||
public:
|
||||
/// Launches a background thread to collect metrics with interval
|
||||
void startCollectMetric(size_t collect_interval_milliseconds_);
|
||||
|
||||
/// Stop background thread. Call before shutdown.
|
||||
void stopCollectMetric();
|
||||
|
||||
private:
|
||||
void metricThreadFunction();
|
||||
|
||||
ThreadFromGlobalPool metric_flush_thread;
|
||||
size_t collect_interval_milliseconds;
|
||||
std::atomic<bool> is_shutdown_metric_thread{false};
|
||||
};
|
||||
|
||||
}
|
@ -166,6 +166,7 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
throw Exception("Empty mutation commands list", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
const ColumnsDescription & columns_desc = storage->getColumns();
|
||||
const IndicesDescription & indices_desc = storage->getIndices();
|
||||
NamesAndTypesList all_columns = columns_desc.getAllPhysical();
|
||||
|
||||
NameSet updated_columns;
|
||||
@ -175,9 +176,10 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
updated_columns.insert(kv.first);
|
||||
}
|
||||
|
||||
/// We need to know which columns affect which MATERIALIZED columns to recalculate them if dependencies
|
||||
/// are updated.
|
||||
/// We need to know which columns affect which MATERIALIZED columns and data skipping indices
|
||||
/// to recalculate them if dependencies are updated.
|
||||
std::unordered_map<String, Names> column_to_affected_materialized;
|
||||
NameSet affected_indices_columns;
|
||||
if (!updated_columns.empty())
|
||||
{
|
||||
for (const auto & column : columns_desc)
|
||||
@ -193,24 +195,40 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
}
|
||||
}
|
||||
for (const auto & index : indices_desc.indices)
|
||||
{
|
||||
auto query = index->expr->clone();
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns);
|
||||
const auto required_columns = syntax_result->requiredSourceColumns();
|
||||
|
||||
for (const String & dependency : required_columns)
|
||||
{
|
||||
if (updated_columns.count(dependency))
|
||||
{
|
||||
affected_indices_columns.insert(std::cbegin(required_columns), std::cend(required_columns));
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
validateUpdateColumns(storage, updated_columns, column_to_affected_materialized);
|
||||
}
|
||||
|
||||
/// First, break a sequence of commands into stages.
|
||||
stages.emplace_back(context);
|
||||
for (const auto & command : commands)
|
||||
{
|
||||
if (!stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
|
||||
if (command.type == MutationCommand::DELETE)
|
||||
{
|
||||
if (stages.empty() || !stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
|
||||
auto negated_predicate = makeASTFunction("not", command.predicate->clone());
|
||||
stages.back().filters.push_back(negated_predicate);
|
||||
}
|
||||
else if (command.type == MutationCommand::UPDATE)
|
||||
{
|
||||
if (stages.empty() || !stages.back().column_to_updated.empty())
|
||||
stages.emplace_back(context);
|
||||
if (stages.size() == 1) /// First stage only supports filtering and can't update columns.
|
||||
stages.emplace_back(context);
|
||||
|
||||
@ -251,35 +269,84 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (command.type == MutationCommand::MATERIALIZE_INDEX)
|
||||
{
|
||||
auto it = std::find_if(
|
||||
std::cbegin(indices_desc.indices), std::end(indices_desc.indices),
|
||||
[&](const std::shared_ptr<ASTIndexDeclaration> & index)
|
||||
{
|
||||
return index->name == command.index_name;
|
||||
});
|
||||
if (it == std::cend(indices_desc.indices))
|
||||
throw Exception("Unknown index: " + command.index_name, ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
auto query = (*it)->expr->clone();
|
||||
auto syntax_result = SyntaxAnalyzer(context).analyze(query, all_columns);
|
||||
const auto required_columns = syntax_result->requiredSourceColumns();
|
||||
affected_indices_columns.insert(std::cbegin(required_columns), std::cend(required_columns));
|
||||
}
|
||||
else
|
||||
throw Exception("Unknown mutation command type: " + DB::toString<int>(command.type), ErrorCodes::UNKNOWN_MUTATION_COMMAND);
|
||||
}
|
||||
|
||||
/// Next, for each stage calculate columns changed by this and previous stages.
|
||||
for (size_t i = 0; i < stages.size(); ++i)
|
||||
if (!affected_indices_columns.empty())
|
||||
{
|
||||
if (!stages[i].filters.empty())
|
||||
if (!stages.empty())
|
||||
{
|
||||
std::vector<Stage> stages_copy;
|
||||
/// Copy all filled stages except index calculation stage.
|
||||
for (const auto &stage : stages)
|
||||
{
|
||||
stages_copy.emplace_back(context);
|
||||
stages_copy.back().column_to_updated = stage.column_to_updated;
|
||||
stages_copy.back().output_columns = stage.output_columns;
|
||||
stages_copy.back().filters = stage.filters;
|
||||
}
|
||||
auto first_stage_header = prepareInterpreterSelect(stages_copy, /* dry_run = */ true)->getSampleBlock();
|
||||
auto in = std::make_shared<NullBlockInputStream>(first_stage_header);
|
||||
updated_header = std::make_unique<Block>(addStreamsForLaterStages(stages_copy, in)->getHeader());
|
||||
}
|
||||
/// Special step to recalculate affected indices.
|
||||
stages.emplace_back(context);
|
||||
for (const auto & column : affected_indices_columns)
|
||||
stages.back().column_to_updated.emplace(
|
||||
column, std::make_shared<ASTIdentifier>(column));
|
||||
}
|
||||
|
||||
interpreter_select = prepareInterpreterSelect(stages, dry_run);
|
||||
|
||||
is_prepared = true;
|
||||
}
|
||||
|
||||
std::unique_ptr<InterpreterSelectQuery> MutationsInterpreter::prepareInterpreterSelect(std::vector<Stage> & prepared_stages, bool dry_run)
|
||||
{
|
||||
NamesAndTypesList all_columns = storage->getColumns().getAllPhysical();
|
||||
|
||||
/// Next, for each stage calculate columns changed by this and previous stages.
|
||||
for (size_t i = 0; i < prepared_stages.size(); ++i)
|
||||
{
|
||||
if (!prepared_stages[i].filters.empty())
|
||||
{
|
||||
for (const auto & column : all_columns)
|
||||
stages[i].output_columns.insert(column.name);
|
||||
prepared_stages[i].output_columns.insert(column.name);
|
||||
continue;
|
||||
}
|
||||
|
||||
if (i > 0)
|
||||
stages[i].output_columns = stages[i - 1].output_columns;
|
||||
prepared_stages[i].output_columns = prepared_stages[i - 1].output_columns;
|
||||
|
||||
if (stages[i].output_columns.size() < all_columns.size())
|
||||
if (prepared_stages[i].output_columns.size() < all_columns.size())
|
||||
{
|
||||
for (const auto & kv : stages[i].column_to_updated)
|
||||
stages[i].output_columns.insert(kv.first);
|
||||
for (const auto & kv : prepared_stages[i].column_to_updated)
|
||||
prepared_stages[i].output_columns.insert(kv.first);
|
||||
}
|
||||
}
|
||||
|
||||
/// Now, calculate `expressions_chain` for each stage except the first.
|
||||
/// Do it backwards to propagate information about columns required as input for a stage to the previous stage.
|
||||
for (size_t i = stages.size() - 1; i > 0; --i)
|
||||
for (size_t i = prepared_stages.size() - 1; i > 0; --i)
|
||||
{
|
||||
auto & stage = stages[i];
|
||||
auto & stage = prepared_stages[i];
|
||||
|
||||
ASTPtr all_asts = std::make_shared<ASTExpressionList>();
|
||||
|
||||
@ -317,7 +384,7 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
for (const auto & kv : stage.column_to_updated)
|
||||
{
|
||||
actions_chain.getLastActions()->add(ExpressionAction::copyColumn(
|
||||
kv.second->getColumnName(), kv.first, /* can_replace = */ true));
|
||||
kv.second->getColumnName(), kv.first, /* can_replace = */ true));
|
||||
}
|
||||
}
|
||||
|
||||
@ -329,7 +396,7 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
|
||||
/// Propagate information about columns needed as input.
|
||||
for (const auto & column : actions_chain.steps.front().actions->getRequiredColumnsWithTypes())
|
||||
stages[i - 1].output_columns.insert(column.name);
|
||||
prepared_stages[i - 1].output_columns.insert(column.name);
|
||||
}
|
||||
|
||||
/// Execute first stage as a SELECT statement.
|
||||
@ -337,36 +404,34 @@ void MutationsInterpreter::prepare(bool dry_run)
|
||||
auto select = std::make_shared<ASTSelectQuery>();
|
||||
|
||||
select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
|
||||
for (const auto & column_name : stages[0].output_columns)
|
||||
for (const auto & column_name : prepared_stages[0].output_columns)
|
||||
select->select()->children.push_back(std::make_shared<ASTIdentifier>(column_name));
|
||||
|
||||
if (!stages[0].filters.empty())
|
||||
if (!prepared_stages[0].filters.empty())
|
||||
{
|
||||
ASTPtr where_expression;
|
||||
if (stages[0].filters.size() == 1)
|
||||
where_expression = stages[0].filters[0];
|
||||
if (prepared_stages[0].filters.size() == 1)
|
||||
where_expression = prepared_stages[0].filters[0];
|
||||
else
|
||||
{
|
||||
auto coalesced_predicates = std::make_shared<ASTFunction>();
|
||||
coalesced_predicates->name = "and";
|
||||
coalesced_predicates->arguments = std::make_shared<ASTExpressionList>();
|
||||
coalesced_predicates->children.push_back(coalesced_predicates->arguments);
|
||||
coalesced_predicates->arguments->children = stages[0].filters;
|
||||
coalesced_predicates->arguments->children = prepared_stages[0].filters;
|
||||
where_expression = std::move(coalesced_predicates);
|
||||
}
|
||||
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression));
|
||||
}
|
||||
|
||||
interpreter_select = std::make_unique<InterpreterSelectQuery>(select, context, storage, SelectQueryOptions().analyze(dry_run).ignoreLimits());
|
||||
|
||||
is_prepared = true;
|
||||
return std::make_unique<InterpreterSelectQuery>(select, context, storage, SelectQueryOptions().analyze(dry_run).ignoreLimits());
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(BlockInputStreamPtr in) const
|
||||
BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::vector<Stage> & prepared_stages, BlockInputStreamPtr in) const
|
||||
{
|
||||
for (size_t i_stage = 1; i_stage < stages.size(); ++i_stage)
|
||||
for (size_t i_stage = 1; i_stage < prepared_stages.size(); ++i_stage)
|
||||
{
|
||||
const Stage & stage = stages[i_stage];
|
||||
const Stage & stage = prepared_stages[i_stage];
|
||||
|
||||
for (size_t i = 0; i < stage.expressions_chain.steps.size(); ++i)
|
||||
{
|
||||
@ -398,14 +463,22 @@ void MutationsInterpreter::validate()
|
||||
prepare(/* dry_run = */ true);
|
||||
Block first_stage_header = interpreter_select->getSampleBlock();
|
||||
BlockInputStreamPtr in = std::make_shared<NullBlockInputStream>(first_stage_header);
|
||||
addStreamsForLaterStages(in)->getHeader();
|
||||
addStreamsForLaterStages(stages, in)->getHeader();
|
||||
}
|
||||
|
||||
BlockInputStreamPtr MutationsInterpreter::execute()
|
||||
{
|
||||
prepare(/* dry_run = */ false);
|
||||
BlockInputStreamPtr in = interpreter_select->execute().in;
|
||||
return addStreamsForLaterStages(in);
|
||||
auto result_stream = addStreamsForLaterStages(stages, in);
|
||||
if (!updated_header)
|
||||
updated_header = std::make_unique<Block>(result_stream->getHeader());
|
||||
return result_stream;
|
||||
}
|
||||
|
||||
const Block & MutationsInterpreter::getUpdatedHeader() const
|
||||
{
|
||||
return *updated_header;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
|
||||
class Context;
|
||||
|
||||
/// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs)
|
||||
/// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs, MATERIALIZEs)
|
||||
/// to this data.
|
||||
class MutationsInterpreter
|
||||
{
|
||||
@ -30,13 +30,19 @@ public:
|
||||
/// Return false if the data isn't going to be changed by mutations.
|
||||
bool isStorageTouchedByMutations() const;
|
||||
|
||||
/// The resulting stream will return blocks containing changed columns only.
|
||||
/// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices.
|
||||
BlockInputStreamPtr execute();
|
||||
|
||||
/// Only changed columns.
|
||||
const Block & getUpdatedHeader() const;
|
||||
|
||||
private:
|
||||
void prepare(bool dry_run);
|
||||
|
||||
BlockInputStreamPtr addStreamsForLaterStages(BlockInputStreamPtr in) const;
|
||||
struct Stage;
|
||||
|
||||
std::unique_ptr<InterpreterSelectQuery> prepareInterpreterSelect(std::vector<Stage> & prepared_stages, bool dry_run);
|
||||
BlockInputStreamPtr addStreamsForLaterStages(const std::vector<Stage> & prepared_stages, BlockInputStreamPtr in) const;
|
||||
|
||||
private:
|
||||
StoragePtr storage;
|
||||
@ -78,6 +84,7 @@ private:
|
||||
};
|
||||
|
||||
std::unique_ptr<InterpreterSelectQuery> interpreter_select;
|
||||
std::unique_ptr<Block> updated_header;
|
||||
std::vector<Stage> stages;
|
||||
bool is_prepared = false; /// Has the sequence of stages been prepared.
|
||||
};
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include "DatabaseAndTableWithAlias.h"
|
||||
#include "ExpressionAnalyzer.h"
|
||||
#include <Parsers/ASTSelectQuery.h>
|
||||
#include <map>
|
||||
|
||||
|
@ -103,7 +103,6 @@ void NO_INLINE Aggregator::executeSpecialized(
|
||||
size_t rows,
|
||||
ColumnRawPtrs & key_columns,
|
||||
AggregateColumns & aggregate_columns,
|
||||
StringRefs & keys,
|
||||
bool no_more_keys,
|
||||
AggregateDataPtr overflow_row) const
|
||||
{
|
||||
@ -111,10 +110,10 @@ void NO_INLINE Aggregator::executeSpecialized(
|
||||
|
||||
if (!no_more_keys)
|
||||
executeSpecializedCase<false, Method, AggregateFunctionsList>(
|
||||
method, state, aggregates_pool, rows, key_columns, aggregate_columns, keys, overflow_row);
|
||||
method, state, aggregates_pool, rows, aggregate_columns, overflow_row);
|
||||
else
|
||||
executeSpecializedCase<true, Method, AggregateFunctionsList>(
|
||||
method, state, aggregates_pool, rows, key_columns, aggregate_columns, keys, overflow_row);
|
||||
method, state, aggregates_pool, rows, aggregate_columns, overflow_row);
|
||||
}
|
||||
|
||||
#pragma GCC diagnostic push
|
||||
@ -126,9 +125,7 @@ void NO_INLINE Aggregator::executeSpecializedCase(
|
||||
typename Method::State & state,
|
||||
Arena * aggregates_pool,
|
||||
size_t rows,
|
||||
ColumnRawPtrs & /*key_columns*/,
|
||||
AggregateColumns & aggregate_columns,
|
||||
StringRefs & /*keys*/,
|
||||
AggregateDataPtr overflow_row) const
|
||||
{
|
||||
/// For all rows.
|
||||
@ -184,20 +181,10 @@ void NO_INLINE Aggregator::executeSpecializedWithoutKey(
|
||||
AggregateColumns & aggregate_columns,
|
||||
Arena * arena) const
|
||||
{
|
||||
/// Optimization in the case of a single aggregate function `count`.
|
||||
AggregateFunctionCount * agg_count = params.aggregates_size == 1
|
||||
? typeid_cast<AggregateFunctionCount *>(aggregate_functions[0])
|
||||
: nullptr;
|
||||
|
||||
if (agg_count)
|
||||
agg_count->addDelta(res, rows);
|
||||
else
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
AggregateFunctionsList::forEach(AggregateFunctionsUpdater(
|
||||
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, res, i, arena));
|
||||
}
|
||||
AggregateFunctionsList::forEach(AggregateFunctionsUpdater(
|
||||
aggregate_functions, offsets_of_aggregate_states, aggregate_columns, res, i, arena));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -12,7 +12,6 @@ class Join;
|
||||
using JoinPtr = std::shared_ptr<Join>;
|
||||
|
||||
class InterpreterSelectWithUnionQuery;
|
||||
struct AnalyzedJoin;
|
||||
|
||||
|
||||
/// Information on what to do when executing a subquery in the [GLOBAL] IN/JOIN section.
|
||||
|
@ -1,7 +1,9 @@
|
||||
#include <Core/Settings.h>
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
#include <Interpreters/SyntaxAnalyzer.h>
|
||||
#include <Interpreters/InJoinSubqueriesPreprocessor.h>
|
||||
#include <Interpreters/LogicalExpressionsOptimizer.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <Interpreters/QueryAliasesVisitor.h>
|
||||
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
|
||||
#include <Interpreters/ArrayJoinedColumnsVisitor.h>
|
||||
@ -14,6 +16,8 @@
|
||||
#include <Interpreters/ExternalDictionaries.h>
|
||||
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
|
||||
#include <Interpreters/RequiredSourceColumnsVisitor.h>
|
||||
#include <Interpreters/GetAggregatesVisitor.h>
|
||||
#include <Interpreters/ExpressionActions.h> /// getSmallestColumn()
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -27,10 +31,8 @@
|
||||
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
#include <functional>
|
||||
|
||||
@ -46,6 +48,7 @@ namespace ErrorCodes
|
||||
extern const int EMPTY_LIST_OF_COLUMNS_QUERIED;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int UNKNOWN_IDENTIFIER;
|
||||
extern const int EXPECTED_ALL_OR_ANY;
|
||||
}
|
||||
|
||||
NameSet removeDuplicateColumns(NamesAndTypesList & columns)
|
||||
@ -485,6 +488,27 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
|
||||
}
|
||||
}
|
||||
|
||||
void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness)
|
||||
{
|
||||
const ASTTablesInSelectQueryElement * node = select_query.join();
|
||||
if (!node)
|
||||
return;
|
||||
|
||||
auto & table_join = const_cast<ASTTablesInSelectQueryElement *>(node)->table_join->as<ASTTableJoin &>();
|
||||
|
||||
if (table_join.strictness == ASTTableJoin::Strictness::Unspecified &&
|
||||
table_join.kind != ASTTableJoin::Kind::Cross)
|
||||
{
|
||||
if (join_default_strictness == JoinStrictness::ANY)
|
||||
table_join.strictness = ASTTableJoin::Strictness::Any;
|
||||
else if (join_default_strictness == JoinStrictness::ALL)
|
||||
table_join.strictness = ASTTableJoin::Strictness::All;
|
||||
else
|
||||
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty",
|
||||
DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
|
||||
}
|
||||
}
|
||||
|
||||
/// Find the columns that are obtained by JOIN.
|
||||
void collectJoinedColumns(AnalyzedJoin & analyzed_join, const ASTSelectQuery & select_query, const NameSet & source_columns,
|
||||
const Aliases & aliases, bool join_use_nulls)
|
||||
@ -558,6 +582,30 @@ void checkJoin(const ASTTablesInSelectQueryElement * join)
|
||||
ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
std::vector<const ASTFunction *> getAggregates(const ASTPtr & query)
|
||||
{
|
||||
if (const auto * select_query = query->as<ASTSelectQuery>())
|
||||
{
|
||||
/// There can not be aggregate functions inside the WHERE and PREWHERE.
|
||||
if (select_query->where())
|
||||
assertNoAggregates(select_query->where(), "in WHERE");
|
||||
if (select_query->prewhere())
|
||||
assertNoAggregates(select_query->prewhere(), "in PREWHERE");
|
||||
|
||||
GetAggregatesVisitor::Data data;
|
||||
GetAggregatesVisitor(data).visit(query);
|
||||
|
||||
/// There can not be other aggregate functions within the aggregate functions.
|
||||
for (const ASTFunction * node : data.aggregates)
|
||||
for (auto & arg : node->arguments->children)
|
||||
assertNoAggregates(arg, "inside another aggregate function");
|
||||
return data.aggregates;
|
||||
}
|
||||
else
|
||||
assertNoAggregates(query, "in wrong place");
|
||||
return {};
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
/// Calculate which columns are required to execute the expression.
|
||||
@ -837,9 +885,11 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
|
||||
/// Push the predicate expression down to the subqueries.
|
||||
result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
|
||||
|
||||
setJoinStrictness(*select_query, settings.join_default_strictness);
|
||||
collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases, settings.join_use_nulls);
|
||||
}
|
||||
|
||||
result.aggregates = getAggregates(query);
|
||||
result.collectUsedColumns(query, additional_source_columns);
|
||||
return std::make_shared<const SyntaxAnalyzerResult>(result);
|
||||
}
|
||||
|
@ -10,6 +10,8 @@ namespace DB
|
||||
|
||||
NameSet removeDuplicateColumns(NamesAndTypesList & columns);
|
||||
|
||||
class ASTFunction;
|
||||
|
||||
struct SyntaxAnalyzerResult
|
||||
{
|
||||
StoragePtr storage;
|
||||
@ -22,6 +24,7 @@ struct SyntaxAnalyzerResult
|
||||
NamesAndTypesList columns_added_by_join;
|
||||
|
||||
Aliases aliases;
|
||||
std::vector<const ASTFunction *> aggregates;
|
||||
|
||||
/// Which column is needed to be ARRAY-JOIN'ed to get the specified.
|
||||
/// For example, for `SELECT s.v ... ARRAY JOIN a AS s` will get "s.v" -> "a.v".
|
||||
|
@ -4,6 +4,7 @@
|
||||
#include <Interpreters/PartLog.h>
|
||||
#include <Interpreters/TextLog.h>
|
||||
#include <Interpreters/TraceLog.h>
|
||||
#include <Interpreters/MetricLog.h>
|
||||
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
@ -48,6 +49,13 @@ SystemLogs::SystemLogs(Context & global_context, const Poco::Util::AbstractConfi
|
||||
part_log = createSystemLog<PartLog>(global_context, "system", "part_log", config, "part_log");
|
||||
trace_log = createSystemLog<TraceLog>(global_context, "system", "trace_log", config, "trace_log");
|
||||
text_log = createSystemLog<TextLog>(global_context, "system", "text_log", config, "text_log");
|
||||
metric_log = createSystemLog<MetricLog>(global_context, "system", "metric_log", config, "metric_log");
|
||||
|
||||
if (metric_log)
|
||||
{
|
||||
size_t collect_interval_milliseconds = config.getUInt64("metric_log.collect_interval_milliseconds");
|
||||
metric_log->startCollectMetric(collect_interval_milliseconds);
|
||||
}
|
||||
|
||||
part_log_database = config.getString("part_log.database", "system");
|
||||
}
|
||||
@ -70,6 +78,11 @@ void SystemLogs::shutdown()
|
||||
trace_log->shutdown();
|
||||
if (text_log)
|
||||
text_log->shutdown();
|
||||
if (metric_log)
|
||||
{
|
||||
metric_log->stopCollectMetric();
|
||||
metric_log->shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -61,6 +61,7 @@ class QueryThreadLog;
|
||||
class PartLog;
|
||||
class TextLog;
|
||||
class TraceLog;
|
||||
class MetricLog;
|
||||
|
||||
/// System logs should be destroyed in destructor of the last Context and before tables,
|
||||
/// because SystemLog destruction makes insert query while flushing data into underlying tables
|
||||
@ -76,6 +77,7 @@ struct SystemLogs
|
||||
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
|
||||
std::shared_ptr<TraceLog> trace_log; /// Used to log traces from query profiler
|
||||
std::shared_ptr<TextLog> text_log; /// Used to log all text messages.
|
||||
std::shared_ptr<MetricLog> metric_log; /// Used to log all metrics.
|
||||
|
||||
String part_log_database;
|
||||
};
|
||||
|
@ -39,7 +39,7 @@ Block TextLogElement::createBlock()
|
||||
|
||||
{std::move(priority_datatype), "level"},
|
||||
{std::make_shared<DataTypeString>(), "query_id"},
|
||||
{std::make_shared<DataTypeString>(), "logger_name"},
|
||||
{std::make_shared<DataTypeLowCardinality>(std::make_shared<DataTypeString>()), "logger_name"},
|
||||
{std::make_shared<DataTypeString>(), "message"},
|
||||
|
||||
{std::make_shared<DataTypeUInt32>(), "revision"},
|
||||
|
@ -278,12 +278,14 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A
|
||||
{
|
||||
bool has_password = config.has(config_elem + ".password");
|
||||
bool has_password_sha256_hex = config.has(config_elem + ".password_sha256_hex");
|
||||
bool has_password_double_sha1_hex = config.has(config_elem + ".password_double_sha1_hex");
|
||||
|
||||
if (has_password && has_password_sha256_hex)
|
||||
throw Exception("Both fields 'password' and 'password_sha256_hex' are specified for user " + name + ". Must be only one of them.", ErrorCodes::BAD_ARGUMENTS);
|
||||
if (has_password + has_password_sha256_hex + has_password_double_sha1_hex > 1)
|
||||
throw Exception("More than one field of 'password', 'password_sha256_hex', 'password_double_sha1_hex' is used to specify password for user " + name + ". Must be only one of them.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!has_password && !has_password_sha256_hex)
|
||||
throw Exception("Either 'password' or 'password_sha256_hex' must be specified for user " + name + ".", ErrorCodes::BAD_ARGUMENTS);
|
||||
if (!has_password && !has_password_sha256_hex && !has_password_double_sha1_hex)
|
||||
throw Exception("Either 'password' or 'password_sha256_hex' or 'password_double_sha1_hex' must be specified for user " + name + ".", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (has_password)
|
||||
password = config.getString(config_elem + ".password");
|
||||
@ -296,6 +298,14 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A
|
||||
throw Exception("password_sha256_hex for user " + name + " has length " + toString(password_sha256_hex.size()) + " but must be exactly 64 symbols.", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
if (has_password_double_sha1_hex)
|
||||
{
|
||||
password_double_sha1_hex = Poco::toLower(config.getString(config_elem + ".password_double_sha1_hex"));
|
||||
|
||||
if (password_double_sha1_hex.size() != 40)
|
||||
throw Exception("password_double_sha1_hex for user " + name + " has length " + toString(password_double_sha1_hex.size()) + " but must be exactly 40 symbols.", ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
|
||||
profile = config.getString(config_elem + ".profile");
|
||||
quota = config.getString(config_elem + ".quota");
|
||||
|
||||
|
@ -56,6 +56,7 @@ struct User
|
||||
/// Required password. Could be stored in plaintext or in SHA256.
|
||||
String password;
|
||||
String password_sha256_hex;
|
||||
String password_double_sha1_hex;
|
||||
|
||||
String profile;
|
||||
String quota;
|
||||
|
@ -1,14 +1,15 @@
|
||||
#include <Interpreters/UsersManager.h>
|
||||
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/String.h>
|
||||
#include "config_core.h"
|
||||
#include <Common/Exception.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include <IO/HexWriteBuffer.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <common/logger_useful.h>
|
||||
#include "config_core.h"
|
||||
#include <Poco/Net/IPAddress.h>
|
||||
#include <Poco/SHA1Engine.h>
|
||||
#include <Poco/String.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#if USE_SSL
|
||||
# include <openssl/sha.h>
|
||||
#endif
|
||||
@ -93,6 +94,21 @@ UserPtr UsersManager::authorizeAndGetUser(
|
||||
throw DB::Exception("SHA256 passwords support is disabled, because ClickHouse was built without SSL library", DB::ErrorCodes::SUPPORT_IS_DISABLED);
|
||||
#endif
|
||||
}
|
||||
else if (!it->second->password_double_sha1_hex.empty())
|
||||
{
|
||||
Poco::SHA1Engine engine;
|
||||
engine.update(password);
|
||||
const auto & first_sha1 = engine.digest();
|
||||
|
||||
/// If it was MySQL compatibility server, then first_sha1 already contains double SHA1.
|
||||
if (Poco::SHA1Engine::digestToHex(first_sha1) == it->second->password_double_sha1_hex)
|
||||
return it->second;
|
||||
|
||||
engine.update(first_sha1.data(), first_sha1.size());
|
||||
|
||||
if (Poco::SHA1Engine::digestToHex(engine.digest()) != it->second->password_double_sha1_hex)
|
||||
on_wrong_password();
|
||||
}
|
||||
else if (password != it->second->password)
|
||||
{
|
||||
on_wrong_password();
|
||||
|
@ -56,7 +56,7 @@ target_link_libraries (expression_analyzer PRIVATE dbms clickhouse_storages_syst
|
||||
add_check(expression_analyzer)
|
||||
|
||||
add_executable (users users.cpp)
|
||||
target_link_libraries (users PRIVATE dbms clickhouse_common_config ${Boost_FILESYSTEM_LIBRARY})
|
||||
target_link_libraries (users PRIVATE dbms clickhouse_common_config stdc++fs)
|
||||
|
||||
if (OS_LINUX)
|
||||
add_executable (internal_iotop internal_iotop.cpp)
|
||||
|
@ -56,7 +56,7 @@ int main(int argc, char ** argv)
|
||||
};
|
||||
|
||||
auto syntax_result = SyntaxAnalyzer(context, {}).analyze(ast, columns);
|
||||
ExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
SelectQueryExpressionAnalyzer analyzer(ast, syntax_result, context);
|
||||
ExpressionActionsChain chain(context);
|
||||
analyzer.appendSelect(chain, false);
|
||||
analyzer.appendProjectResult(chain);
|
||||
|
@ -109,8 +109,24 @@ void ASTAlterCommand::formatImpl(
|
||||
else if (type == ASTAlterCommand::DROP_INDEX)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< "DROP INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
<< (clear_index ? "CLEAR " : "DROP ") << "INDEX " << (if_exists ? "IF EXISTS " : "") << (settings.hilite ? hilite_none : "");
|
||||
index->formatImpl(settings, state, frame);
|
||||
if (partition)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : "");
|
||||
partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::MATERIALIZE_INDEX)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str
|
||||
<< "MATERIALIZE INDEX " << (settings.hilite ? hilite_none : "");
|
||||
index->formatImpl(settings, state, frame);
|
||||
if (partition)
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_keyword : "") << indent_str<< " IN PARTITION " << (settings.hilite ? hilite_none : "");
|
||||
partition->formatImpl(settings, state, frame);
|
||||
}
|
||||
}
|
||||
else if (type == ASTAlterCommand::ADD_CONSTRAINT)
|
||||
{
|
||||
|
@ -31,6 +31,7 @@ public:
|
||||
|
||||
ADD_INDEX,
|
||||
DROP_INDEX,
|
||||
MATERIALIZE_INDEX,
|
||||
|
||||
ADD_CONSTRAINT,
|
||||
DROP_CONSTRAINT,
|
||||
@ -71,6 +72,8 @@ public:
|
||||
|
||||
/** The ADD INDEX query stores the name of the index following AFTER.
|
||||
* The DROP INDEX query stores the name for deletion.
|
||||
* The MATERIALIZE INDEX query stores the name of the index to materialize.
|
||||
* The CLEAR INDEX query stores the name of the index to clear.
|
||||
*/
|
||||
ASTPtr index;
|
||||
|
||||
@ -105,9 +108,11 @@ public:
|
||||
|
||||
bool clear_column = false; /// for CLEAR COLUMN (do not drop column from metadata)
|
||||
|
||||
bool if_not_exists = false; /// option for ADD_COLUMN
|
||||
bool clear_index = false; /// for CLEAR INDEX (do not drop index from metadata)
|
||||
|
||||
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||
bool if_not_exists = false; /// option for ADD_COLUMN
|
||||
|
||||
bool if_exists = false; /// option for DROP_COLUMN, MODIFY_COLUMN, COMMENT_COLUMN
|
||||
|
||||
/** For FETCH PARTITION - the path in ZK to the shard, from which to download the partition.
|
||||
*/
|
||||
|
@ -12,7 +12,7 @@ class ASTPartition : public IAST
|
||||
{
|
||||
public:
|
||||
ASTPtr value;
|
||||
StringRef fields_str; /// The extent of comma-separated partition expression fields without parentheses.
|
||||
String fields_str; /// The extent of comma-separated partition expression fields without parentheses.
|
||||
size_t fields_count = 0;
|
||||
|
||||
String id;
|
||||
|
@ -31,6 +31,8 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
|
||||
ParserKeyword s_add_index("ADD INDEX");
|
||||
ParserKeyword s_drop_index("DROP INDEX");
|
||||
ParserKeyword s_clear_index("CLEAR INDEX");
|
||||
ParserKeyword s_materialize_index("MATERIALIZE INDEX");
|
||||
|
||||
ParserKeyword s_add_constraint("ADD CONSTRAINT");
|
||||
ParserKeyword s_drop_constraint("DROP CONSTRAINT");
|
||||
@ -102,6 +104,24 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
command->detach = false;
|
||||
}
|
||||
else if (s_clear_column.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
command->clear_column = true;
|
||||
command->detach = false;
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
{
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
}
|
||||
}
|
||||
else if (s_add_index.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_not_exists.ignore(pos, expected))
|
||||
@ -150,16 +170,32 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
|
||||
command->type = ASTAlterCommand::DROP_CONSTRAINT;
|
||||
command->detach = false;
|
||||
}
|
||||
else if (s_clear_column.ignore(pos, expected))
|
||||
else if (s_clear_index.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->column, expected))
|
||||
if (!parser_name.parse(pos, command->index, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::DROP_COLUMN;
|
||||
command->clear_column = true;
|
||||
command->type = ASTAlterCommand::DROP_INDEX;
|
||||
command->clear_index = true;
|
||||
command->detach = false;
|
||||
|
||||
if (!s_in_partition.ignore(pos, expected))
|
||||
return false;
|
||||
if (!parser_partition.parse(pos, command->partition, expected))
|
||||
return false;
|
||||
}
|
||||
else if (s_materialize_index.ignore(pos, expected))
|
||||
{
|
||||
if (s_if_exists.ignore(pos, expected))
|
||||
command->if_exists = true;
|
||||
|
||||
if (!parser_name.parse(pos, command->index, expected))
|
||||
return false;
|
||||
|
||||
command->type = ASTAlterCommand::MATERIALIZE_INDEX;
|
||||
command->detach = false;
|
||||
|
||||
if (s_in_partition.ignore(pos, expected))
|
||||
|
@ -35,7 +35,7 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
|
||||
size_t fields_count;
|
||||
StringRef fields_str;
|
||||
String fields_str;
|
||||
|
||||
const auto * tuple_ast = value->as<ASTFunction>();
|
||||
if (tuple_ast && tuple_ast->name == "tuple")
|
||||
@ -59,17 +59,17 @@ bool ParserPartition::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (right_paren->type != TokenType::ClosingRoundBracket)
|
||||
return false;
|
||||
|
||||
fields_str = StringRef(left_paren->end, right_paren->begin - left_paren->end);
|
||||
fields_str = String(left_paren->end, right_paren->begin - left_paren->end);
|
||||
}
|
||||
else
|
||||
{
|
||||
fields_count = 1;
|
||||
fields_str = StringRef(begin->begin, pos->begin - begin->begin);
|
||||
fields_str = String(begin->begin, pos->begin - begin->begin);
|
||||
}
|
||||
|
||||
partition->value = value;
|
||||
partition->children.push_back(value);
|
||||
partition->fields_str = fields_str;
|
||||
partition->fields_str = std::move(fields_str);
|
||||
partition->fields_count = fields_count;
|
||||
}
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user