Merge branch 'master' into table-constraints

This commit is contained in:
Gleb Novikov 2019-08-20 12:17:56 +03:00
commit 86fecc7501
230 changed files with 4659 additions and 2036 deletions

2
.gitmodules vendored
View File

@ -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

View File

@ -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")

View File

@ -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.

View File

@ -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)

View File

@ -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 ()

View File

@ -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

@ -1 +1 @@
Subproject commit d85d0e98999cd9e28ceb66645999b4a9ce85370e
Subproject commit c6503d3acc85ca1a7f5e7e38b605d7c9410aac1e

View File

@ -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

View File

@ -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

@ -1 +1 @@
Subproject commit 7a2d304c21549427460428c9039009ef4bbfd899
Subproject commit 6216cc01a107ce149863411ca29013a224f80343

2
contrib/simdjson vendored

@ -1 +1 @@
Subproject commit e3f6322af762213ff2087ce3366bf9541c7fd355
Subproject commit e9be643db5cf1c29a69bc80ee72d220124a9c50e

View File

@ -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)

View File

@ -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 "")

View File

@ -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();
}
}

View File

@ -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)

View File

@ -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;

View File

@ -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

View 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>

View File

@ -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/
-->

View File

@ -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>

View File

@ -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__; }
};

View 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);
}
};

View File

@ -26,6 +26,11 @@ void CurrentThread::updatePerformanceCounters()
current_thread->updatePerformanceCounters();
}
bool CurrentThread::isInitialized()
{
return current_thread;
}
ThreadStatus & CurrentThread::get()
{
if (unlikely(!current_thread))

View File

@ -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();

View File

@ -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

View File

@ -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>;

View File

@ -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;

View File

@ -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.

View File

@ -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();

View 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;
}

View 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();

View File

@ -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 (...)

View File

@ -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)

View File

@ -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

View File

@ -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)

View File

@ -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()))
)
),
);
}

View File

@ -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.

View File

@ -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;
};
}
}
}

View File

@ -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;
}

View File

@ -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
{

View File

@ -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; }

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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_);
}

View File

@ -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);

View File

@ -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);

View File

@ -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 = {};

View File

@ -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
);

View File

@ -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);
}

View File

@ -25,6 +25,7 @@ target_link_libraries(clickhouse_functions
PRIVATE
${ZLIB_LIBRARIES}
${Boost_FILESYSTEM_LIBRARY}
${CMAKE_DL_LIBS}
)
if (OPENSSL_CRYPTO_LIBRARY)

View File

@ -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;

View File

@ -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]])

View File

@ -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));

View File

@ -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));

View File

@ -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));
}

View File

@ -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

View 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);
}
}

View File

@ -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);
}

View File

@ -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);

View File

@ -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;
}
}
}

View 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 {}
};
}

View File

@ -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)

View File

@ -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);
}
}

View File

@ -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;
}

View File

@ -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)

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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());

View File

@ -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();

View File

@ -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.

View File

@ -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);

View File

@ -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;
};
}

View 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);
}
}

View File

@ -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>

View File

@ -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
{

View File

@ -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.

View File

@ -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:

View 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__);
}
}
}
}

View 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};
};
}

View File

@ -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;
}
}

View File

@ -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.
};

View File

@ -1,7 +1,6 @@
#pragma once
#include "DatabaseAndTableWithAlias.h"
#include "ExpressionAnalyzer.h"
#include <Parsers/ASTSelectQuery.h>
#include <map>

View File

@ -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));
}
}

View File

@ -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.

View File

@ -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);
}

View File

@ -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".

View File

@ -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();
}
}
}

View File

@ -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;
};

View File

@ -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"},

View File

@ -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");

View File

@ -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;

View File

@ -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();

View File

@ -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)

View File

@ -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);

View File

@ -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)
{

View File

@ -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.
*/

View File

@ -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;

View File

@ -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))

View File

@ -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