Merge branch 'master' into no-finalize-WriteBufferFromOStream

This commit is contained in:
Sema Checherinda 2023-06-27 19:02:27 +02:00 committed by GitHub
commit d6f1e4b527
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
186 changed files with 3132 additions and 2985 deletions

View File

@ -27,6 +27,7 @@ curl https://clickhouse.com/ | sh
* [**ClickHouse Meetup in Boston**](https://www.meetup.com/clickhouse-boston-user-group/events/293913596) - Jul 18
* [**ClickHouse Meetup in NYC**](https://www.meetup.com/clickhouse-new-york-user-group/events/293913441) - Jul 19
* [**ClickHouse Meetup in Toronto**](https://www.meetup.com/clickhouse-toronto-user-group/events/294183127) - Jul 20
* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/294428050/) - Jul 27
Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler <at> clickhouse <dot> com.

View File

@ -7,7 +7,13 @@
/** Returns value `from` converted to type `To` while retaining bit representation.
* `To` and `From` must satisfy `CopyConstructible`.
*
* In contrast to std::bit_cast can cast types of different width.
*
* Note: for signed types of narrower size, the casted result is zero-extended
* instead of sign-extended as with regular static_cast.
* For example, -1 Int8 (represented as 0xFF) bit_casted to UInt64
* gives 255 (represented as 0x00000000000000FF) instead of 0xFFFFFFFFFFFFFFFF
*/
template <typename To, typename From>
std::decay_t<To> bit_cast(const From & from)

View File

@ -1,53 +0,0 @@
//
// ConsoleCertificateHandler.h
//
// Library: NetSSL_OpenSSL
// Package: SSLCore
// Module: ConsoleCertificateHandler
//
// Definition of the ConsoleCertificateHandler class.
//
// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH.
// and Contributors.
//
// SPDX-License-Identifier: BSL-1.0
//
#ifndef NetSSL_ConsoleCertificateHandler_INCLUDED
#define NetSSL_ConsoleCertificateHandler_INCLUDED
#include "Poco/Net/InvalidCertificateHandler.h"
#include "Poco/Net/NetSSL.h"
namespace Poco
{
namespace Net
{
class NetSSL_API ConsoleCertificateHandler : public InvalidCertificateHandler
/// A ConsoleCertificateHandler is invoked whenever an error occurs verifying the certificate.
///
/// The certificate is printed to stdout and the user is asked via console if he wants to accept it.
{
public:
ConsoleCertificateHandler(bool handleErrorsOnServerSide);
/// Creates the ConsoleCertificateHandler.
virtual ~ConsoleCertificateHandler();
/// Destroys the ConsoleCertificateHandler.
void onInvalidCertificate(const void * pSender, VerificationErrorArgs & errorCert);
/// Prints the certificate to stdout and waits for user input on the console
/// to decide if a certificate should be accepted/rejected.
};
}
} // namespace Poco::Net
#endif // NetSSL_ConsoleCertificateHandler_INCLUDED

View File

@ -85,7 +85,7 @@ namespace Net
/// </options>
/// </privateKeyPassphraseHandler>
/// <invalidCertificateHandler>
/// <name>ConsoleCertificateHandler</name>
/// <name>RejectCertificateHandler</name>
/// </invalidCertificateHandler>
/// <cacheSessions>true|false</cacheSessions>
/// <sessionIdContext>someString</sessionIdContext> <!-- server only -->
@ -186,7 +186,7 @@ namespace Net
///
/// Valid initialization code would be:
/// SharedPtr<PrivateKeyPassphraseHandler> pConsoleHandler = new KeyConsoleHandler;
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new ConsoleCertificateHandler;
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new RejectCertificateHandler;
/// Context::Ptr pContext = new Context(Context::SERVER_USE, "any.pem", "any.pem", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH");
/// SSLManager::instance().initializeServer(pConsoleHandler, pInvalidCertHandler, pContext);
@ -203,7 +203,7 @@ namespace Net
///
/// Valid initialization code would be:
/// SharedPtr<PrivateKeyPassphraseHandler> pConsoleHandler = new KeyConsoleHandler;
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new ConsoleCertificateHandler;
/// SharedPtr<InvalidCertificateHandler> pInvalidCertHandler = new RejectCertificateHandler;
/// Context::Ptr pContext = new Context(Context::CLIENT_USE, "", "", "rootcert.pem", Context::VERIFY_RELAXED, 9, false, "ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH");
/// SSLManager::instance().initializeClient(pConsoleHandler, pInvalidCertHandler, pContext);

View File

@ -13,7 +13,6 @@
#include "Poco/Net/CertificateHandlerFactoryMgr.h"
#include "Poco/Net/ConsoleCertificateHandler.h"
#include "Poco/Net/AcceptCertificateHandler.h"
#include "Poco/Net/RejectCertificateHandler.h"
@ -24,7 +23,6 @@ namespace Net {
CertificateHandlerFactoryMgr::CertificateHandlerFactoryMgr()
{
setFactory("ConsoleCertificateHandler", new CertificateHandlerFactoryImpl<ConsoleCertificateHandler>());
setFactory("AcceptCertificateHandler", new CertificateHandlerFactoryImpl<AcceptCertificateHandler>());
setFactory("RejectCertificateHandler", new CertificateHandlerFactoryImpl<RejectCertificateHandler>());
}

View File

@ -1,53 +0,0 @@
//
// ConsoleCertificateHandler.cpp
//
// Library: NetSSL_OpenSSL
// Package: SSLCore
// Module: ConsoleCertificateHandler
//
// Copyright (c) 2006-2009, Applied Informatics Software Engineering GmbH.
// and Contributors.
//
// SPDX-License-Identifier: BSL-1.0
//
#include "Poco/Net/ConsoleCertificateHandler.h"
#include <iostream>
namespace Poco {
namespace Net {
ConsoleCertificateHandler::ConsoleCertificateHandler(bool server): InvalidCertificateHandler(server)
{
}
ConsoleCertificateHandler::~ConsoleCertificateHandler()
{
}
void ConsoleCertificateHandler::onInvalidCertificate(const void*, VerificationErrorArgs& errorCert)
{
const X509Certificate& aCert = errorCert.certificate();
std::cout << "\n";
std::cout << "WARNING: Certificate verification failed\n";
std::cout << "----------------------------------------\n";
std::cout << "Issuer Name: " << aCert.issuerName() << "\n";
std::cout << "Subject Name: " << aCert.subjectName() << "\n\n";
std::cout << "The certificate yielded the error: " << errorCert.errorMessage() << "\n\n";
std::cout << "The error occurred in the certificate chain at position " << errorCert.errorDepth() << "\n";
std::cout << "Accept the certificate (y,n)? ";
char c = 0;
std::cin >> c;
if (c == 'y' || c == 'Y')
errorCert.setIgnoreError(true);
else
errorCert.setIgnoreError(false);
}
} } // namespace Poco::Net

View File

@ -46,7 +46,7 @@ const std::string SSLManager::CFG_PREFER_SERVER_CIPHERS("preferServerCiphers");
const std::string SSLManager::CFG_DELEGATE_HANDLER("privateKeyPassphraseHandler.name");
const std::string SSLManager::VAL_DELEGATE_HANDLER("KeyConsoleHandler");
const std::string SSLManager::CFG_CERTIFICATE_HANDLER("invalidCertificateHandler.name");
const std::string SSLManager::VAL_CERTIFICATE_HANDLER("ConsoleCertificateHandler");
const std::string SSLManager::VAL_CERTIFICATE_HANDLER("RejectCertificateHandler");
const std::string SSLManager::CFG_SERVER_PREFIX("openSSL.server.");
const std::string SSLManager::CFG_CLIENT_PREFIX("openSSL.client.");
const std::string SSLManager::CFG_CACHE_SESSIONS("cacheSessions");

View File

@ -31,12 +31,12 @@ endif()
set (CMAKE_CXX_STANDARD 17)
set(ARROW_VERSION "6.0.1")
set(ARROW_VERSION "11.0.0")
string(REGEX MATCH "^[0-9]+\\.[0-9]+\\.[0-9]+" ARROW_BASE_VERSION "${ARROW_VERSION}")
set(ARROW_VERSION_MAJOR "6")
set(ARROW_VERSION_MAJOR "11")
set(ARROW_VERSION_MINOR "0")
set(ARROW_VERSION_PATCH "1")
set(ARROW_VERSION_PATCH "0")
if(ARROW_VERSION_MAJOR STREQUAL "0")
# Arrow 0.x.y => SO version is "x", full SO version is "x.y.0"
@ -514,6 +514,10 @@ if (SANITIZE STREQUAL "undefined")
target_compile_options(_arrow PRIVATE -fno-sanitize=undefined)
endif ()
# Define Thrift version for parquet (we use 0.16.0)
add_definitions(-DPARQUET_THRIFT_VERSION_MAJOR=0)
add_definitions(-DPARQUET_THRIFT_VERSION_MINOR=16)
# === tools
set(TOOLS_DIR "${ClickHouse_SOURCE_DIR}/contrib/arrow/cpp/tools/parquet")

View File

@ -14,6 +14,13 @@ LEFT_SERVER_PORT=9001
# patched version
RIGHT_SERVER_PORT=9002
# abort_conf -- abort if some options is not recognized
# abort -- abort if something is not right in the env (i.e. per-cpu arenas does not work)
# narenas -- set them explicitly to avoid disabling per-cpu arena in env
# that returns different number of CPUs for some of the following
# _SC_NPROCESSORS_ONLN/_SC_NPROCESSORS_CONF/sched_getaffinity
export MALLOC_CONF="abort_conf:true,abort:true,narenas:$(nproc --all)"
function wait_for_server # port, pid
{
for _ in {1..60}
@ -109,10 +116,6 @@ function restart
while pkill -f clickhouse-serv ; do echo . ; sleep 1 ; done
echo all killed
# Change the jemalloc settings here.
# https://github.com/jemalloc/jemalloc/wiki/Getting-Started
export MALLOC_CONF="confirm_conf:true"
set -m # Spawn servers in their own process groups
local left_server_opts=(
@ -147,8 +150,6 @@ function restart
set +m
unset MALLOC_CONF
wait_for_server $LEFT_SERVER_PORT $left_pid
echo left ok

View File

@ -86,7 +86,7 @@ def process_test_log(log_path, broken_tests):
test_name,
"NOT_FAILED",
test_time,
["This test passed. Update broken_tests.txt.\n"],
["This test passed. Update analyzer_tech_debt.txt.\n"],
)
)
else:
@ -205,7 +205,7 @@ if __name__ == "__main__":
parser.add_argument("--in-results-dir", default="/test_output/")
parser.add_argument("--out-results-file", default="/test_output/test_results.tsv")
parser.add_argument("--out-status-file", default="/test_output/check_status.tsv")
parser.add_argument("--broken-tests", default="/broken_tests.txt")
parser.add_argument("--broken-tests", default="/analyzer_tech_debt.txt")
args = parser.parse_args()
broken_tests = list()

View File

@ -1602,7 +1602,7 @@ Keys for server/client settings:
- requireTLSv1_2 (default: false) Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
- fips (default: false) Activates OpenSSL FIPS mode. Supported if the librarys OpenSSL version supports FIPS.
- privateKeyPassphraseHandler (default: `KeyConsoleHandler`) Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
- invalidCertificateHandler (default: `ConsoleCertificateHandler`) Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>` .
- invalidCertificateHandler (default: `RejectCertificateHandler`) Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>RejectCertificateHandler</name> </invalidCertificateHandler>` .
- disableProtocols (default: "") Protocols that are not allowed to use.
- preferServerCiphers (default: false) Preferred server ciphers on the client.

View File

@ -19,8 +19,19 @@ Standard aggregate functions:
- [stddevSamp](/docs/en/sql-reference/aggregate-functions/reference/stddevsamp.md)
- [varPop](/docs/en/sql-reference/aggregate-functions/reference/varpop.md)
- [varSamp](/docs/en/sql-reference/aggregate-functions/reference/varsamp.md)
- [corr](./corr.md)
- [covarPop](/docs/en/sql-reference/aggregate-functions/reference/covarpop.md)
- [covarSamp](/docs/en/sql-reference/aggregate-functions/reference/covarsamp.md)
- [entropy](./entropy.md)
- [exponentialMovingAverage](./exponentialmovingaverage.md)
- [intervalLengthSum](./intervalLengthSum.md)
- [kolmogorovSmirnovTest](./kolmogorovsmirnovtest.md)
- [mannwhitneyutest](./mannwhitneyutest.md)
- [median](./median.md)
- [rankCorr](./rankCorr.md)
- [sumKahan](./sumkahan.md)
- [studentTTest](./studentttest.md)
- [welchTTest](./welchttest.md)
ClickHouse-specific aggregate functions:
@ -34,12 +45,15 @@ ClickHouse-specific aggregate functions:
- [avgWeighted](/docs/en/sql-reference/aggregate-functions/reference/avgweighted.md)
- [topK](/docs/en/sql-reference/aggregate-functions/reference/topk.md)
- [topKWeighted](/docs/en/sql-reference/aggregate-functions/reference/topkweighted.md)
- [deltaSum](./deltasum.md)
- [deltaSumTimestamp](./deltasumtimestamp.md)
- [groupArray](/docs/en/sql-reference/aggregate-functions/reference/grouparray.md)
- [groupArrayLast](/docs/en/sql-reference/aggregate-functions/reference/grouparraylast.md)
- [groupUniqArray](/docs/en/sql-reference/aggregate-functions/reference/groupuniqarray.md)
- [groupArrayInsertAt](/docs/en/sql-reference/aggregate-functions/reference/grouparrayinsertat.md)
- [groupArrayMovingAvg](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingavg.md)
- [groupArrayMovingSum](/docs/en/sql-reference/aggregate-functions/reference/grouparraymovingsum.md)
- [groupArraySample](./grouparraysample.md)
- [groupBitAnd](/docs/en/sql-reference/aggregate-functions/reference/groupbitand.md)
- [groupBitOr](/docs/en/sql-reference/aggregate-functions/reference/groupbitor.md)
- [groupBitXor](/docs/en/sql-reference/aggregate-functions/reference/groupbitxor.md)
@ -84,3 +98,9 @@ ClickHouse-specific aggregate functions:
- [theilsU](./theilsu.md)
- [maxIntersections](./maxintersections.md)
- [maxIntersectionsPosition](./maxintersectionsposition.md)
- [meanZTest](./meanztest.md)
- [quantileGK](./quantileGK.md)
- [quantileInterpolatedWeighted](./quantileinterpolatedweighted.md)
- [sparkBar](./sparkbar.md)
- [sumCount](./sumcount.md)

View File

@ -21,6 +21,9 @@ Expressions from `ON` clause and columns from `USING` clause are called “join
## Related Content
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Part 1](https://clickhouse.com/blog/clickhouse-fully-supports-joins)
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 2](https://clickhouse.com/blog/clickhouse-fully-supports-joins-hash-joins-part2)
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 3](https://clickhouse.com/blog/clickhouse-fully-supports-joins-full-sort-partial-merge-part3)
- Blog: [ClickHouse: A Blazingly Fast DBMS with Full SQL Join Support - Under the Hood - Part 4](https://clickhouse.com/blog/clickhouse-fully-supports-joins-direct-join-part4)
## Supported Types of JOIN

View File

@ -1067,7 +1067,7 @@ ClickHouse использует потоки из глобального пул
- requireTLSv1_2 - Требование соединения TLSv1.2. Допустимые значения: `true`, `false`.
- fips - Активация режима OpenSSL FIPS. Поддерживается, если версия OpenSSL, с которой собрана библиотека поддерживает fips.
- privateKeyPassphraseHandler - Класс (подкласс PrivateKeyPassphraseHandler)запрашивающий кодовую фразу доступа к секретному ключу. Например, `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>`.
- invalidCertificateHandler - Класс (подкласс CertificateHandler) для подтверждения не валидных сертификатов. Например, `<invalidCertificateHandler> <name>RejectCertificateHandler</name> </invalidCertificateHandler>`.
- disableProtocols - Запрещенные к использованию протоколы.
- preferServerCiphers - Предпочтение серверных шифров на клиенте.

View File

@ -466,7 +466,7 @@ SSL客户端/服务器配置。
- requireTLSv1_2 Require a TLSv1.2 connection. Acceptable values: `true`, `false`.
- fips Activates OpenSSL FIPS mode. Supported if the librarys OpenSSL version supports FIPS.
- privateKeyPassphraseHandler Class (PrivateKeyPassphraseHandler subclass) that requests the passphrase for accessing the private key. For example: `<privateKeyPassphraseHandler>`, `<name>KeyFileHandler</name>`, `<options><password>test</password></options>`, `</privateKeyPassphraseHandler>`.
- invalidCertificateHandler Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>ConsoleCertificateHandler</name> </invalidCertificateHandler>` .
- invalidCertificateHandler Class (a subclass of CertificateHandler) for verifying invalid certificates. For example: `<invalidCertificateHandler> <name>RejectCertificateHandler</name> </invalidCertificateHandler>` .
- disableProtocols Protocols that are not allowed to use.
- preferServerCiphers Preferred server ciphers on the client.

View File

@ -282,6 +282,13 @@
<cacheSessions>true</cacheSessions>
<disableProtocols>sslv2,sslv3</disableProtocols>
<preferServerCiphers>true</preferServerCiphers>
<invalidCertificateHandler>
<!-- The server, in contrast to the client, cannot ask about the certificate interactively.
The only reasonable option is to reject.
-->
<name>RejectCertificateHandler</name>
</invalidCertificateHandler>
</server>
<client> <!-- Used for connecting to https dictionary source and secured Zookeeper communication -->

View File

@ -2383,7 +2383,6 @@ QueryTreeNodePtr QueryAnalyzer::tryResolveTableIdentifierFromDatabaseCatalog(con
auto storage_lock = storage->lockForShare(context->getInitialQueryId(), context->getSettingsRef().lock_acquire_timeout);
auto storage_snapshot = storage->getStorageSnapshot(storage->getInMemoryMetadataPtr(), context);
auto result = std::make_shared<TableNode>(std::move(storage), std::move(storage_lock), std::move(storage_snapshot));
if (is_temporary_table)
result->setTemporaryTableName(table_name);
@ -5210,14 +5209,26 @@ ProjectionNames QueryAnalyzer::resolveFunction(QueryTreeNodePtr & node, Identifi
const auto & second_argument_constant_literal = second_argument_constant_node->getValue();
const auto & second_argument_constant_type = second_argument_constant_node->getResultType();
auto set = makeSetForConstantValue(first_argument_constant_type,
const auto & settings = scope.context->getSettingsRef();
auto result_block = getSetElementsForConstantValue(first_argument_constant_type,
second_argument_constant_literal,
second_argument_constant_type,
scope.context->getSettingsRef());
settings.transform_null_in);
SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};
auto set = std::make_shared<Set>(size_limits_for_set, 0, settings.transform_null_in);
set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName());
set->insertFromBlock(result_block.getColumnsWithTypeAndName());
set->finishInsert();
auto future_set = std::make_shared<FutureSetFromStorage>(std::move(set));
/// Create constant set column for constant folding
auto column_set = ColumnSet::create(1, FutureSet(std::move(set)));
auto column_set = ColumnSet::create(1, std::move(future_set));
argument_columns[1].column = ColumnConst::create(std::move(column_set), 1);
}

View File

@ -118,7 +118,7 @@ Block createBlockFromCollection(const Collection & collection, const DataTypes &
}
SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings)
Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in)
{
DataTypes set_element_types = {expression_type};
const auto * lhs_tuple_type = typeid_cast<const DataTypeTuple *>(expression_type.get());
@ -135,9 +135,6 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
size_t lhs_type_depth = getCompoundTypeDepth(*expression_type);
size_t rhs_type_depth = getCompoundTypeDepth(*value_type);
SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};
bool tranform_null_in = settings.transform_null_in;
Block result_block;
if (lhs_type_depth == rhs_type_depth)
@ -145,7 +142,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
/// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc.
Array array{value};
result_block = createBlockFromCollection(array, set_element_types, tranform_null_in);
result_block = createBlockFromCollection(array, set_element_types, transform_null_in);
}
else if (lhs_type_depth + 1 == rhs_type_depth)
{
@ -154,9 +151,9 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
WhichDataType rhs_which_type(value_type);
if (rhs_which_type.isArray())
result_block = createBlockFromCollection(value.get<const Array &>(), set_element_types, tranform_null_in);
result_block = createBlockFromCollection(value.get<const Array &>(), set_element_types, transform_null_in);
else if (rhs_which_type.isTuple())
result_block = createBlockFromCollection(value.get<const Tuple &>(), set_element_types, tranform_null_in);
result_block = createBlockFromCollection(value.get<const Tuple &>(), set_element_types, transform_null_in);
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Unsupported type at the right-side of IN. Expected Array or Tuple. Actual {}",
@ -170,13 +167,7 @@ SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field
value_type->getName());
}
auto set = std::make_shared<Set>(size_limits_for_set, true /*fill_set_elements*/, tranform_null_in);
set->setHeader(result_block.cloneEmpty().getColumnsWithTypeAndName());
set->insertFromBlock(result_block.getColumnsWithTypeAndName());
set->finishInsert();
return set;
return result_block;
}
}

View File

@ -12,19 +12,12 @@ namespace DB
class Set;
using SetPtr = std::shared_ptr<Set>;
/** Make set for constant part of IN subquery.
/** Get set elements for constant part of IN subquery.
* Throws exception if parameters are not valid for IN function.
*
* Example: SELECT id FROM test_table WHERE id IN (1, 2, 3, 4);
* Example: SELECT id FROM test_table WHERE id IN ((1, 2), (3, 4));
*
* @param expression_type - type of first argument of function IN.
* @param value - constant value of second argument of function IN.
* @param value_type - type of second argument of function IN.
* @param settings - query settings.
*
* @return SetPtr for constant value.
*/
SetPtr makeSetForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, const Settings & settings);
Block getSetElementsForConstantValue(const DataTypePtr & expression_type, const Field & value, const DataTypePtr & value_type, bool transform_null_in);
}

View File

@ -21,7 +21,7 @@ class ColumnSet final : public COWHelper<IColumnDummy, ColumnSet>
private:
friend class COWHelper<IColumnDummy, ColumnSet>;
ColumnSet(size_t s_, FutureSet data_) : data(std::move(data_)) { s = s_; }
ColumnSet(size_t s_, FutureSetPtr data_) : data(std::move(data_)) { s = s_; }
ColumnSet(const ColumnSet &) = default;
public:
@ -29,13 +29,13 @@ public:
TypeIndex getDataType() const override { return TypeIndex::Set; }
MutableColumnPtr cloneDummy(size_t s_) const override { return ColumnSet::create(s_, data); }
ConstSetPtr getData() const { if (!data.isReady()) return nullptr; return data.get(); }
FutureSetPtr getData() const { return data; }
// Used only for debugging, making it DUMPABLE
Field operator[](size_t) const override { return {}; }
private:
FutureSet data;
FutureSetPtr data;
};
}

View File

@ -21,12 +21,6 @@
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
}
/** Variants for searching a substring in a string.
* In most cases, performance is less than Volnitsky (see Volnitsky.h).
*/
@ -811,15 +805,22 @@ class TokenSearcher : public StringSearcherBase
size_t needle_size;
public:
template <typename CharT>
requires (sizeof(CharT) == 1)
static bool isValidNeedle(const CharT * needle_, size_t needle_size_)
{
return std::none_of(needle_, needle_ + needle_size_, isTokenSeparator);
}
template <typename CharT>
requires (sizeof(CharT) == 1)
TokenSearcher(const CharT * needle_, size_t needle_size_)
: searcher(needle_, needle_size_)
, needle_size(needle_size_)
{
if (std::any_of(needle_, needle_ + needle_size_, isTokenSeparator))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters");
/// The caller is responsible for calling isValidNeedle()
chassert(isValidNeedle(needle_, needle_size_));
}
template <typename CharT>
@ -880,6 +881,7 @@ using ASCIICaseSensitiveStringSearcher = impl::StringSearcher<true, true>;
using ASCIICaseInsensitiveStringSearcher = impl::StringSearcher<false, true>;
using UTF8CaseSensitiveStringSearcher = impl::StringSearcher<true, false>;
using UTF8CaseInsensitiveStringSearcher = impl::StringSearcher<false, false>;
using ASCIICaseSensitiveTokenSearcher = impl::TokenSearcher<ASCIICaseSensitiveStringSearcher>;
using ASCIICaseInsensitiveTokenSearcher = impl::TokenSearcher<ASCIICaseInsensitiveStringSearcher>;

View File

@ -406,8 +406,7 @@ public:
/// And also adding from the end guarantees that we will find first occurrence because we will lookup bigger offsets first.
for (auto i = static_cast<ssize_t>(needle_size - sizeof(VolnitskyTraits::Ngram)); i >= 0; --i)
{
bool ok = VolnitskyTraits::putNGram<CaseSensitive, ASCII>(
needle + i, static_cast<int>(i + 1), needle, needle_size, callback);
bool ok = VolnitskyTraits::putNGram<CaseSensitive, ASCII>(needle + i, static_cast<int>(i + 1), needle, needle_size, callback);
/** `putNGramUTF8CaseInsensitive` does not work if characters with lower and upper cases
* are represented by different number of bytes or code points.

View File

@ -141,7 +141,7 @@ class IColumn;
M(Bool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
M(Bool, allow_suspicious_fixed_string_types, false, "In CREATE TABLE statement allows creating columns of type FixedString(n) with n > 256. FixedString with length >= 256 is suspicious and most likely indicates misusage", 0) \
M(Bool, allow_suspicious_indices, false, "Reject primary/secondary indexes and sorting keys with identical expressions", 0) \
M(Bool, compile_expressions, true, "Compile some scalar functions and operators to native code.", 0) \
M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \
M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \
M(Bool, compile_aggregate_expressions, true, "Compile aggregate functions to native code. This feature has a bug and should not be used.", 0) \
M(UInt64, min_count_to_compile_aggregate_expression, 3, "The number of identical aggregate expressions before they are JIT-compiled", 0) \

View File

@ -22,7 +22,7 @@ public:
bool isParametric() const override { return true; }
// Used for expressions analysis.
MutableColumnPtr createColumn() const override { return ColumnSet::create(0, FutureSet{}); }
MutableColumnPtr createColumn() const override { return ColumnSet::create(0, nullptr); }
// Used only for debugging, making it DUMPABLE
Field getDefault() const override { return Tuple(); }

View File

@ -875,10 +875,11 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
LOG_TEST(
log,
"Current read type: {}, read offset: {}, impl offset: {}, file segment: {}",
"Current read type: {}, read offset: {}, impl offset: {}, impl position: {}, file segment: {}",
toString(read_type),
file_offset_of_buffer_end,
implementation_buffer->getFileOffsetOfBufferEnd(),
implementation_buffer->getPosition(),
file_segment.getInfoForLog());
chassert(current_read_range.left <= file_offset_of_buffer_end);
@ -937,7 +938,8 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
// We don't support implementation_buffer implementations that use nextimpl_working_buffer_offset.
chassert(implementation_buffer->position() == implementation_buffer->buffer().begin());
size = implementation_buffer->buffer().size();
if (result)
size = implementation_buffer->buffer().size();
LOG_TEST(
log,
@ -951,15 +953,21 @@ bool CachedOnDiskReadBufferFromFile::nextImplStep()
ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheBytes, size);
ProfileEvents::increment(ProfileEvents::CachedReadBufferReadFromCacheMicroseconds, elapsed);
const size_t new_file_offset = file_offset_of_buffer_end + size;
const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true);
if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset)
if (result)
{
auto file_segment_path = file_segment.getPathInLocalCache();
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Read unexpected size. File size: {}, file path: {}, file segment info: {}",
fs::file_size(file_segment_path), file_segment_path, file_segment.getInfoForLog());
const size_t new_file_offset = file_offset_of_buffer_end + size;
const size_t file_segment_write_offset = file_segment.getCurrentWriteOffset(true);
if (new_file_offset > file_segment.range().right + 1 || new_file_offset > file_segment_write_offset)
{
auto file_segment_path = file_segment.getPathInLocalCache();
throw Exception(
ErrorCodes::LOGICAL_ERROR, "Read unexpected size. "
"File size: {}, file segment path: {}, impl size: {}, impl path: {}"
"file segment info: {}",
fs::file_size(file_segment_path), file_segment_path,
implementation_buffer->getFileSize(), implementation_buffer->getFileName(),
file_segment.getInfoForLog());
}
}
}
else

View File

@ -14,6 +14,7 @@ public:
virtual bool isValid() = 0;
virtual RelativePathWithMetadata current() = 0;
virtual RelativePathsWithMetadata currentBatch() = 0;
virtual std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() = 0;
virtual size_t getAccumulatedSize() const = 0;
virtual ~IObjectStorageIterator() = default;
@ -53,6 +54,11 @@ public:
return batch;
}
virtual std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() override
{
return std::nullopt;
}
size_t getAccumulatedSize() const override
{
return batch.size();

View File

@ -100,6 +100,22 @@ RelativePathsWithMetadata IObjectStorageIteratorAsync::currentBatch()
return current_batch;
}
std::optional<RelativePathsWithMetadata> IObjectStorageIteratorAsync::getCurrrentBatchAndScheduleNext()
{
std::lock_guard lock(mutex);
if (!is_initialized)
nextBatch();
if (current_batch_iterator != current_batch.end())
{
auto temp_current_batch = current_batch;
nextBatch();
return temp_current_batch;
}
return std::nullopt;
}
size_t IObjectStorageIteratorAsync::getAccumulatedSize() const
{
return accumulated_size.load(std::memory_order_relaxed);

View File

@ -27,6 +27,7 @@ public:
RelativePathWithMetadata current() override;
RelativePathsWithMetadata currentBatch() override;
size_t getAccumulatedSize() const override;
std::optional<RelativePathsWithMetadata> getCurrrentBatchAndScheduleNext() override;
~IObjectStorageIteratorAsync() override
{
@ -48,7 +49,7 @@ protected:
bool is_initialized{false};
bool is_finished{false};
mutable std::mutex mutex;
mutable std::recursive_mutex mutex;
ThreadPool list_objects_pool;
ThreadPoolCallbackRunner<BatchAndHasNext> list_objects_scheduler;
std::future<BatchAndHasNext> outcome_future;

View File

@ -322,7 +322,6 @@ struct ToTimeImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToDateImpl;
};
@ -394,7 +393,6 @@ struct ToStartOfSecondImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -442,7 +440,6 @@ struct ToStartOfMillisecondImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -486,7 +483,6 @@ struct ToStartOfMicrosecondImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -524,7 +520,6 @@ struct ToStartOfNanosecondImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -723,28 +718,6 @@ struct ToYearImpl
return time_zone.toYear(DayNum(d));
}
static inline constexpr bool hasPreimage() { return true; }
static inline RangeOrNull getPreimage(const IDataType & type, const Field & point)
{
if (point.getType() != Field::Types::UInt64) return std::nullopt;
auto year = point.get<UInt64>();
if (year < DATE_LUT_MIN_YEAR || year >= DATE_LUT_MAX_YEAR) return std::nullopt;
const DateLUTImpl & date_lut = DateLUT::instance();
auto start_time = date_lut.makeDateTime(year, 1, 1, 0, 0, 0);
auto end_time = date_lut.addYears(start_time, 1);
if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type))
return {std::make_pair(Field(start_time), Field(end_time))};
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64",
type.getName(), name);
}
using FactorTransform = ZeroTransform;
};
@ -818,7 +791,6 @@ struct ToQuarterImpl
{
return time_zone.toQuarter(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToStartOfYearImpl;
};
@ -843,7 +815,6 @@ struct ToMonthImpl
{
return time_zone.toMonth(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToStartOfYearImpl;
};
@ -869,7 +840,6 @@ struct ToDayOfMonthImpl
return time_zone.toDayOfMonth(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToStartOfMonthImpl;
};
@ -917,7 +887,6 @@ struct ToDayOfYearImpl
{
return time_zone.toDayOfYear(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToStartOfYearImpl;
};
@ -942,7 +911,6 @@ struct ToHourImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToDateImpl;
};
@ -971,7 +939,6 @@ struct TimezoneOffsetImpl
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToTimeImpl;
};
@ -995,7 +962,6 @@ struct ToMinuteImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToStartOfHourImpl;
};
@ -1020,7 +986,6 @@ struct ToSecondImpl
{
throwDateTimeIsNotSupported(name);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToStartOfMinuteImpl;
};
@ -1045,7 +1010,6 @@ struct ToISOYearImpl
{
return time_zone.toISOYear(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1102,7 +1066,6 @@ struct ToISOWeekImpl
{
return time_zone.toISOWeek(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ToISOYearImpl;
};
@ -1145,7 +1108,6 @@ struct ToRelativeYearNumImpl
{
return time_zone.toYear(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1177,7 +1139,6 @@ struct ToRelativeQuarterNumImpl
{
return time_zone.toRelativeQuarterNum(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1209,7 +1170,6 @@ struct ToRelativeMonthNumImpl
{
return time_zone.toRelativeMonthNum(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1241,7 +1201,6 @@ struct ToRelativeWeekNumImpl
{
return time_zone.toRelativeWeekNum(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1273,7 +1232,6 @@ struct ToRelativeDayNumImpl
{
return static_cast<DayNum>(d);
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1311,7 +1269,6 @@ struct ToRelativeHourNumImpl
else
return static_cast<UInt32>(time_zone.toRelativeHourNum(DayNum(d)));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1343,7 +1300,6 @@ struct ToRelativeMinuteNumImpl
{
return static_cast<UInt32>(time_zone.toRelativeMinuteNum(DayNum(d)));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1372,7 +1328,6 @@ struct ToRelativeSecondNumImpl
{
return static_cast<UInt32>(time_zone.fromDayNum(DayNum(d)));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1397,31 +1352,6 @@ struct ToYYYYMMImpl
{
return time_zone.toNumYYYYMM(DayNum(d));
}
static inline constexpr bool hasPreimage() { return true; }
static inline RangeOrNull getPreimage(const IDataType & type, const Field & point)
{
if (point.getType() != Field::Types::UInt64) return std::nullopt;
auto year_month = point.get<UInt64>();
auto year = year_month / 100;
auto month = year_month % 100;
if (year < DATE_LUT_MIN_YEAR || year > DATE_LUT_MAX_YEAR || month < 1 || month > 12 || (year == DATE_LUT_MAX_YEAR && month == 12))
return std::nullopt;
const DateLUTImpl & date_lut = DateLUT::instance();
auto start_time = date_lut.makeDateTime(year, month, 1, 0, 0, 0);
auto end_time = date_lut.addMonths(start_time, 1);
if (isDateOrDate32(type) || isDateTime(type) || isDateTime64(type))
return {std::make_pair(Field(start_time), Field(end_time))};
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}. Should be Date, Date32, DateTime or DateTime64",
type.getName(), name);
}
using FactorTransform = ZeroTransform;
};
@ -1446,7 +1376,6 @@ struct ToYYYYMMDDImpl
{
return time_zone.toNumYYYYMMDD(DayNum(d));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};
@ -1471,7 +1400,6 @@ struct ToYYYYMMDDhhmmssImpl
{
return time_zone.toNumYYYYMMDDhhmmss(time_zone.toDate(DayNum(d)));
}
static inline constexpr bool hasPreimage() { return false; }
using FactorTransform = ZeroTransform;
};

View File

@ -7,7 +7,6 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NOT_IMPLEMENTED;
}
/// See DateTimeTransforms.h
@ -84,18 +83,6 @@ public:
arguments[0].type->getName(), this->getName());
}
bool hasInformationAboutPreimage() const override { return Transform::hasPreimage(); }
RangeOrNull getPreimage(const IDataType & type, const Field & point) const override
{
if constexpr (Transform::hasPreimage())
return Transform::getPreimage(type, point);
else
throw Exception(ErrorCodes::NOT_IMPLEMENTED,
"Function {} has no information about its preimage",
Transform::name);
}
};
}

View File

@ -402,34 +402,6 @@ struct SipHash128ReferenceImpl
static constexpr bool use_int_hash_for_pods = false;
};
struct SipHash128ReferenceKeyedImpl
{
static constexpr auto name = "sipHash128ReferenceKeyed";
using ReturnType = UInt128;
using Key = impl::SipHashKey;
static Key parseKey(const ColumnWithTypeAndName & key) { return impl::parseSipHashKey(key); }
static UInt128 applyKeyed(const Key & key, const char * begin, size_t size)
{
return sipHash128ReferenceKeyed(key.key0, key.key1, begin, size);
}
static UInt128 combineHashesKeyed(const Key & key, UInt128 h1, UInt128 h2)
{
#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__
UInt128 tmp;
reverseMemcpy(&tmp, &h1, sizeof(UInt128));
h1 = tmp;
reverseMemcpy(&tmp, &h2, sizeof(UInt128));
h2 = tmp;
#endif
UInt128 hashes[] = {h1, h2};
return applyKeyed(key, reinterpret_cast<const char *>(hashes), 2 * sizeof(UInt128));
}
static constexpr bool use_int_hash_for_pods = false;
};
/** Why we need MurmurHash2?
* MurmurHash2 is an outdated hash function, superseded by MurmurHash3 and subsequently by CityHash, xxHash, HighwayHash.
@ -1737,7 +1709,6 @@ using FunctionSHA512 = FunctionStringHashFixedString<SHA512Impl>;
using FunctionSipHash128 = FunctionAnyHash<SipHash128Impl>;
using FunctionSipHash128Keyed = FunctionAnyHash<SipHash128KeyedImpl, true, SipHash128KeyedImpl::Key>;
using FunctionSipHash128Reference = FunctionAnyHash<SipHash128ReferenceImpl>;
using FunctionSipHash128ReferenceKeyed = FunctionAnyHash<SipHash128ReferenceKeyedImpl, true, SipHash128ReferenceKeyedImpl::Key>;
using FunctionCityHash64 = FunctionAnyHash<ImplCityHash64>;
using FunctionFarmFingerprint64 = FunctionAnyHash<ImplFarmFingerprint64>;
using FunctionFarmHash64 = FunctionAnyHash<ImplFarmHash64>;

View File

@ -20,11 +20,6 @@ REGISTER_FUNCTION(Hashing)
.examples{{"hash", "SELECT hex(sipHash128Reference('foo', '\\x01', 3))", ""}},
.categories{"Hash"}
});
factory.registerFunction<FunctionSipHash128ReferenceKeyed>(FunctionDocumentation{
.description="Same as [sipHash128Reference](#hash_functions-siphash128reference) but additionally takes an explicit key argument instead of using a fixed key.",
.examples{{"hash", "SELECT hex(sipHash128ReferenceKeyed((506097522914230528, 1084818905618843912),'foo', '\\x01', 3));", ""}},
.categories{"Hash"}
});
factory.registerFunction<FunctionCityHash64>();
factory.registerFunction<FunctionFarmFingerprint64>();
factory.registerFunction<FunctionFarmHash64>();

View File

@ -1,6 +1,7 @@
#pragma once
#include <Columns/ColumnString.h>
#include <Common/StringSearcher.h>
#include <Core/ColumnNumbers.h>
@ -9,6 +10,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
@ -44,51 +46,47 @@ struct HasTokenImpl
const UInt8 * const end = haystack_data.data() + haystack_data.size();
const UInt8 * pos = begin;
try
if (!ASCIICaseSensitiveTokenSearcher::isValidNeedle(pattern.data(), pattern.size()))
{
/// Parameter `pattern` is supposed to be a literal of letters and/or numbers.
/// Otherwise, an exception from the constructor of `TokenSearcher` is thrown.
/// If no exception is thrown at that point, then no further error cases may occur.
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
if (res_null)
std::ranges::fill(res_null->getData(), false);
/// The current index in the array of strings.
size_t i = 0;
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Let's determine which index it refers to.
while (begin + haystack_offsets[i] <= pos)
{
res[i] = negate;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + pattern.size() < begin + haystack_offsets[i])
res[i] = !negate;
else
res[i] = negate;
pos = begin + haystack_offsets[i];
++i;
}
/// Tail, in which there can be no substring.
if (i < res.size())
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
}
catch (...)
{
if (!res_null)
throw;
else
{
std::ranges::fill(res, 0);
std::ranges::fill(res_null->getData(), true);
return;
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Needle must not contain whitespace or separator characters");
}
TokenSearcher searcher(pattern.data(), pattern.size(), end - pos);
if (res_null)
std::ranges::fill(res_null->getData(), false);
/// The current index in the array of strings.
size_t i = 0;
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Let's determine which index it refers to.
while (begin + haystack_offsets[i] <= pos)
{
res[i] = negate;
++i;
}
/// We check that the entry does not pass through the boundaries of strings.
if (pos + pattern.size() < begin + haystack_offsets[i])
res[i] = !negate;
else
res[i] = negate;
pos = begin + haystack_offsets[i];
++i;
}
/// Tail, in which there can be no substring.
if (i < res.size())
memset(&res[i], negate, (res.size() - i) * sizeof(res[0]));
}
template <typename... Args>

View File

@ -2,8 +2,6 @@
#include <Core/ColumnNumbers.h>
#include <Core/ColumnsWithTypeAndName.h>
#include <Core/Field.h>
#include <Core/ValuesWithType.h>
#include <Core/Names.h>
#include <Core/IResolvedFunction.h>
#include <Common/Exception.h>
@ -13,6 +11,11 @@
#include <memory>
#if USE_EMBEDDED_COMPILER
# include <Core/ValuesWithType.h>
#endif
/// This file contains user interface for functions.
namespace llvm
@ -32,8 +35,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
/// A left-closed and right-open interval representing the preimage of a function.
using RangeOrNull = std::optional<std::pair<Field, Field>>;
class Field;
/// The simplest executable object.
/// Motivation:
@ -231,12 +233,6 @@ public:
*/
virtual bool hasInformationAboutMonotonicity() const { return false; }
/** Lets you know if the function has its definition of preimage.
* This is used to work with predicate optimizations, where the comparison between
* f(x) and a constant c could be converted to the comparison between x and f's preimage [b, e).
*/
virtual bool hasInformationAboutPreimage() const { return false; }
struct ShortCircuitSettings
{
/// Should we enable lazy execution for the first argument of short-circuit function?
@ -290,14 +286,6 @@ public:
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName());
}
/** Get the preimage of a function in the form of a left-closed and right-open interval. Call only if hasInformationAboutPreimage.
* std::nullopt might be returned if the point (a single value) is invalid for this function.
*/
virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName());
}
};
using FunctionBasePtr = std::shared_ptr<const IFunctionBase>;
@ -487,17 +475,12 @@ public:
virtual bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const = 0;
virtual bool hasInformationAboutMonotonicity() const { return false; }
virtual bool hasInformationAboutPreimage() const { return false; }
using Monotonicity = IFunctionBase::Monotonicity;
virtual Monotonicity getMonotonicityForRange(const IDataType & /*type*/, const Field & /*left*/, const Field & /*right*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its monotonicity", getName());
}
virtual RangeOrNull getPreimage(const IDataType & /*type*/, const Field & /*point*/) const
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Function {} has no information about its preimage", getName());
}
/// For non-variadic functions, return number of arguments; otherwise return zero (that should be ignored).
virtual size_t getNumberOfArguments() const = 0;

View File

@ -90,17 +90,10 @@ public:
bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); }
bool hasInformationAboutPreimage() const override { return function->hasInformationAboutPreimage(); }
Monotonicity getMonotonicityForRange(const IDataType & type, const Field & left, const Field & right) const override
{
return function->getMonotonicityForRange(type, left, right);
}
RangeOrNull getPreimage(const IDataType & type, const Field & point) const override
{
return function->getPreimage(type, point);
}
private:
std::shared_ptr<IFunction> function;
DataTypes arguments;

View File

@ -139,7 +139,7 @@ struct LowerUpperUTF8Impl
/// In case partial buffer was passed (due to SSE optimization)
/// we cannot convert it with current src_end, but we may have more
/// bytes to convert and eventually got correct symbol.
if (partial && src_sequence_length > static_cast<size_t>(src_end-src))
if (partial && src_sequence_length > static_cast<size_t>(src_end - src))
return false;
auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src);
@ -181,7 +181,9 @@ private:
#ifdef __SSE2__
static constexpr auto bytes_sse = sizeof(__m128i);
const auto * src_end_sse = src + (src_end - src) / bytes_sse * bytes_sse;
/// If we are before this position, we can still read at least bytes_sse.
const auto * src_end_sse = src_end - bytes_sse + 1;
/// SSE2 packed comparison operate on signed types, hence compare (c < 0) instead of (c > 0x7f)
const auto v_zero = _mm_setzero_si128();
@ -227,9 +229,11 @@ private:
{
/// UTF-8
/// Find the offset of the next string after src
size_t offset_from_begin = src - begin;
while (offset_from_begin >= *offset_it)
++offset_it;
/// Do not allow one row influence another (since row may have invalid sequence, and break the next)
const UInt8 * row_end = begin + *offset_it;
chassert(row_end >= src);
@ -247,8 +251,9 @@ private:
}
}
/// Find which offset src has now
while (offset_it != offsets.end() && static_cast<size_t>(src - begin) >= *offset_it)
/// Find the offset of the next string after src
size_t offset_from_begin = src - begin;
while (offset_it != offsets.end() && offset_from_begin >= *offset_it)
++offset_it;
#endif

View File

@ -59,14 +59,13 @@ protected:
void setResult(StringRefs & result, const Dwarf::LocationInfo & location, const std::vector<Dwarf::SymbolizedFrame> & inline_frames) const override
{
appendLocationToResult(result, location, nullptr);
for (const auto & inline_frame : inline_frames)
appendLocationToResult(result, inline_frame.location, &inline_frame);
}
private:
inline ALWAYS_INLINE void appendLocationToResult(StringRefs & result, const Dwarf::LocationInfo & location, const Dwarf::SymbolizedFrame * frame) const
private:
void appendLocationToResult(StringRefs & result, const Dwarf::LocationInfo & location, const Dwarf::SymbolizedFrame * frame) const
{
const char * arena_begin = nullptr;
WriteBufferFromArena out(cache.arena, arena_begin);
@ -83,6 +82,7 @@ private:
}
result.emplace_back(out.complete());
out.finalize();
}
};

View File

@ -16,18 +16,18 @@ struct NameHasTokenOrNull
static constexpr auto name = "hasTokenOrNull";
};
using FunctionHasToken = DB::FunctionsStringSearch<DB::HasTokenImpl<NameHasToken, DB::VolnitskyCaseSensitiveToken, false>>;
using FunctionHasTokenOrNull = DB::
FunctionsStringSearch<DB::HasTokenImpl<NameHasTokenOrNull, DB::VolnitskyCaseSensitiveToken, false>, DB::ExecutionErrorPolicy::Null>;
using FunctionHasToken
= FunctionsStringSearch<HasTokenImpl<NameHasToken, VolnitskyCaseSensitiveToken, false>>;
using FunctionHasTokenOrNull
= FunctionsStringSearch<HasTokenImpl<NameHasTokenOrNull, VolnitskyCaseSensitiveToken, false>, ExecutionErrorPolicy::Null>;
REGISTER_FUNCTION(HasToken)
{
factory.registerFunction<FunctionHasToken>(FunctionDocumentation
{.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, DB::FunctionFactory::CaseSensitive);
{.description="Performs lookup of needle in haystack using tokenbf_v1 index."}, FunctionFactory::CaseSensitive);
factory.registerFunction<FunctionHasTokenOrNull>(FunctionDocumentation
{.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."},
DB::FunctionFactory::CaseSensitive);
{.description="Performs lookup of needle in haystack using tokenbf_v1 index. Returns null if needle is ill-formed."}, FunctionFactory::CaseSensitive);
}
}

View File

@ -17,10 +17,9 @@ struct NameHasTokenCaseInsensitiveOrNull
};
using FunctionHasTokenCaseInsensitive
= DB::FunctionsStringSearch<DB::HasTokenImpl<NameHasTokenCaseInsensitive, DB::VolnitskyCaseInsensitiveToken, false>>;
using FunctionHasTokenCaseInsensitiveOrNull = DB::FunctionsStringSearch<
DB::HasTokenImpl<NameHasTokenCaseInsensitiveOrNull, DB::VolnitskyCaseInsensitiveToken, false>,
DB::ExecutionErrorPolicy::Null>;
= FunctionsStringSearch<HasTokenImpl<NameHasTokenCaseInsensitive, VolnitskyCaseInsensitiveToken, false>>;
using FunctionHasTokenCaseInsensitiveOrNull
= FunctionsStringSearch<HasTokenImpl<NameHasTokenCaseInsensitiveOrNull, VolnitskyCaseInsensitiveToken, false>, ExecutionErrorPolicy::Null>;
REGISTER_FUNCTION(HasTokenCaseInsensitive)
{

View File

@ -122,9 +122,13 @@ public:
tuple = typeid_cast<const ColumnTuple *>(materialized_tuple.get());
}
auto set = column_set->getData();
auto future_set = column_set->getData();
if (!future_set)
throw Exception(ErrorCodes::LOGICAL_ERROR, "No Set is passed as the second argument for function '{}'", getName());
auto set = future_set->get();
if (!set)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set passed as the second argument for function '{}'", getName());
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not-ready Set is passed as the second argument for function '{}'", getName());
auto set_types = set->getDataTypes();

View File

@ -5,6 +5,7 @@
#include <Columns/ColumnConst.h>
#include <Columns/ColumnDecimal.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Core/DecimalFunctions.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/getLeastSupertype.h>
@ -13,9 +14,10 @@
#include <Functions/IFunction.h>
#include <Interpreters/castColumn.h>
#include <Interpreters/convertFieldToType.h>
#include <Common/Arena.h>
#include <Common/HashTable/HashMap.h>
#include <Common/typeid_cast.h>
#include <Common/FieldVisitorsAccurateComparison.h>
namespace DB
{
@ -25,32 +27,31 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
extern const int LOGICAL_ERROR;
}
namespace
{
/** transform(x, from_array, to_array[, default]) - convert x according to an explicitly passed match.
*/
/** transform(x, [from...], [to...], default)
* - converts the values according to the explicitly specified mapping.
*
* x - what to transform.
* from - a constant array of values for the transformation.
* to - a constant array of values into which values from `from` must be transformed.
* default - what value to use if x is not equal to any of the values in `from`.
* `from` and `to` - arrays of the same size.
*
* Types:
* transform(T, Array(T), Array(U), U) -> U
*
* transform(x, [from...], [to...])
* - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned.
*
* Types:
* transform(T, Array(T), Array(T)) -> T
*
* Note: the implementation is rather cumbersome.
*/
* - converts the values according to the explicitly specified mapping.
*
* x - what to transform.
* from - a constant array of values for the transformation.
* to - a constant array of values into which values from `from` must be transformed.
* default - what value to use if x is not equal to any of the values in `from`.
* `from` and `to` - arrays of the same size.
*
* Types:
* transform(T, Array(T), Array(U), U) -> U
*
* transform(x, [from...], [to...])
* - if `default` is not specified, then for values of `x` for which there is no corresponding element in `from`, the unchanged value of `x` is returned.
*
* Types:
* transform(T, Array(T), Array(T)) -> T
*
* Note: the implementation is rather cumbersome.
*/
class FunctionTransform : public IFunction
{
public:
@ -79,15 +80,6 @@ namespace
args_size);
const DataTypePtr & type_x = arguments[0];
const auto & type_x_nn = removeNullable(type_x);
if (!type_x_nn->isValueRepresentedByNumber() && !isString(type_x_nn) && !isNothing(type_x_nn))
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Unsupported type {} of first argument "
"of function {}, must be numeric type or Date/DateTime or String",
type_x->getName(),
getName());
const DataTypeArray * type_arr_from = checkAndGetDataType<DataTypeArray>(arguments[1].get());
@ -99,14 +91,16 @@ namespace
const auto type_arr_from_nested = type_arr_from->getNestedType();
if ((type_x->isValueRepresentedByNumber() != type_arr_from_nested->isValueRepresentedByNumber())
|| (isString(type_x) != isString(type_arr_from_nested)))
auto src = tryGetLeastSupertype(DataTypes{type_x, type_arr_from_nested});
if (!src
/// Compatibility with previous versions, that allowed even UInt64 with Int64,
/// regardless of ambiguous conversions.
&& !isNativeNumber(type_x) && !isNativeNumber(type_arr_from_nested))
{
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"First argument and elements of array "
"of second argument of function {} must have compatible types: "
"both numeric or both strings.",
"of the second argument of function {} must have compatible types",
getName());
}
@ -157,8 +151,8 @@ namespace
}
}
ColumnPtr
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
ColumnPtr executeImpl(
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override
{
initialize(arguments, result_type);
@ -172,22 +166,42 @@ namespace
default_non_const = castColumn(arguments[3], result_type);
auto column_result = result_type->createColumn();
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const)
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const)
&& !executeString(in, *column_result, default_non_const))
if (cache.is_empty)
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
return default_non_const
? default_non_const
: castColumn(arguments[0], result_type);
}
else if (cache.table_num_to_idx)
{
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const)
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const)
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const))
{
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
}
}
else if (cache.table_string_to_idx)
{
if (!executeString(in, *column_result, default_non_const))
executeContiguous(in, *column_result, default_non_const);
}
else if (cache.table_anything_to_idx)
{
executeAnything(in, *column_result, default_non_const);
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized");
return column_result;
}
@ -204,6 +218,47 @@ namespace
return impl->execute(args, result_type, input_rows_count);
}
void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const
{
const size_t size = in->size();
const auto & table = *cache.table_anything_to_idx;
column_result.reserve(size);
for (size_t i = 0; i < size; ++i)
{
SipHash hash;
in->updateHashWithValue(i, hash);
const auto * it = table.find(hash.get128());
if (it)
column_result.insertFrom(*cache.to_column, it->getMapped());
else if (cache.default_column)
column_result.insertFrom(*cache.default_column, 0);
else if (default_non_const)
column_result.insertFrom(*default_non_const, i);
else
column_result.insertFrom(*in, i);
}
}
void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const) const
{
const size_t size = in->size();
const auto & table = *cache.table_string_to_idx;
column_result.reserve(size);
for (size_t i = 0; i < size; ++i)
{
const auto * it = table.find(in->getDataAt(i));
if (it)
column_result.insertFrom(*cache.to_column, it->getMapped());
else if (cache.default_column)
column_result.insertFrom(*cache.default_column, 0);
else if (default_non_const)
column_result.insertFrom(*default_non_const, i);
else
column_result.insertFrom(*in, i);
}
}
template <typename T>
bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const) const
{
@ -236,7 +291,7 @@ namespace
{
const auto * it = table.find(bit_cast<UInt64>(pod[i]));
if (it)
column_result.insertFrom(*cache.to_columns, it->getMapped());
column_result.insertFrom(*cache.to_column, it->getMapped());
else if (cache.default_column)
column_result.insertFrom(*cache.default_column, 0);
else if (default_non_const)
@ -259,14 +314,14 @@ namespace
out_offs.resize(size);
auto & out_chars = out->getChars();
const auto * to_col = reinterpret_cast<const ColumnString *>(cache.to_columns.get());
const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get());
const auto & to_chars = to_col->getChars();
const auto & to_offs = to_col->getOffsets();
const auto & table = *cache.table_num_to_idx;
if (cache.default_column)
{
const auto * def = reinterpret_cast<const ColumnString *>(cache.default_column.get());
const auto * def = assert_cast<const ColumnString *>(cache.default_column.get());
const auto & def_chars = def->getChars();
const auto & def_offs = def->getOffsets();
const auto * def_data = def_chars.data();
@ -275,7 +330,7 @@ namespace
}
else
{
const auto * def = reinterpret_cast<const ColumnString *>(default_non_const.get());
const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
const auto & def_chars = def->getChars();
const auto & def_offs = def->getOffsets();
executeNumToStringHelper(table, pod, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size);
@ -340,16 +395,16 @@ namespace
if constexpr (std::is_same_v<ColumnDecimal<Decimal32>, T> || std::is_same_v<ColumnDecimal<Decimal64>, T>)
out_scale = out->getScale();
const auto & to_pod = reinterpret_cast<const T *>(cache.to_columns.get())->getData();
const auto & to_pod = assert_cast<const T *>(cache.to_column.get())->getData();
const auto & table = *cache.table_num_to_idx;
if (cache.default_column)
{
const auto const_def = reinterpret_cast<const T *>(cache.default_column.get())->getData()[0];
const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0];
executeNumToNumHelper(table, pod, out_pod, to_pod, const_def, size, out_scale, out_scale);
}
else if (default_non_const)
{
const auto & nconst_def = reinterpret_cast<const T *>(default_non_const.get())->getData();
const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData();
executeNumToNumHelper(table, pod, out_pod, to_pod, nconst_def, size, out_scale, out_scale);
}
else
@ -423,11 +478,11 @@ namespace
ColumnString::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
const StringRef ref{&data[current_offset], offsets[i] - current_offset};
const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1};
current_offset = offsets[i];
const auto * it = table.find(ref);
if (it)
column_result.insertFrom(*cache.to_columns, it->getMapped());
column_result.insertFrom(*cache.to_column, it->getMapped());
else if (cache.default_column)
column_result.insertFrom(*cache.default_column, 0);
else if (default_non_const)
@ -453,14 +508,14 @@ namespace
out_offs.resize(size);
auto & out_chars = out->getChars();
const auto * to_col = reinterpret_cast<const ColumnString *>(cache.to_columns.get());
const auto * to_col = assert_cast<const ColumnString *>(cache.to_column.get());
const auto & to_chars = to_col->getChars();
const auto & to_offs = to_col->getOffsets();
const auto & table = *cache.table_string_to_idx;
if (cache.default_column)
{
const auto * def = reinterpret_cast<const ColumnString *>(cache.default_column.get());
const auto * def = assert_cast<const ColumnString *>(cache.default_column.get());
const auto & def_chars = def->getChars();
const auto & def_offs = def->getOffsets();
const auto * def_data = def_chars.data();
@ -469,7 +524,7 @@ namespace
}
else if (default_non_const)
{
const auto * def = reinterpret_cast<const ColumnString *>(default_non_const.get());
const auto * def = assert_cast<const ColumnString *>(default_non_const.get());
const auto & def_chars = def->getChars();
const auto & def_offs = def->getOffsets();
executeStringToStringHelper(table, data, offsets, out_chars, out_offs, to_chars, to_offs, def_chars, def_offs, size);
@ -500,7 +555,7 @@ namespace
{
const char8_t * to = nullptr;
size_t to_size = 0;
const StringRef ref{&data[current_offset], offsets[i] - current_offset};
const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1};
current_offset = offsets[i];
const auto * it = table.find(ref);
if (it)
@ -542,16 +597,16 @@ namespace
const size_t size = offsets.size();
out_pod.resize(size);
const auto & to_pod = reinterpret_cast<const T *>(cache.to_columns.get())->getData();
const auto & to_pod = assert_cast<const T *>(cache.to_column.get())->getData();
const auto & table = *cache.table_string_to_idx;
if (cache.default_column)
{
const auto const_def = reinterpret_cast<const T *>(cache.default_column.get())->getData()[0];
const auto const_def = assert_cast<const T *>(cache.default_column.get())->getData()[0];
executeStringToNumHelper(table, data, offsets, out_pod, to_pod, const_def, size);
}
else
{
const auto & nconst_def = reinterpret_cast<const T *>(default_non_const.get())->getData();
const auto & nconst_def = assert_cast<const T *>(default_non_const.get())->getData();
executeStringToNumHelper(table, data, offsets, out_pod, to_pod, nconst_def, size);
}
return true;
@ -570,7 +625,7 @@ namespace
ColumnString::Offset current_offset = 0;
for (size_t i = 0; i < size; ++i)
{
const StringRef ref{&data[current_offset], offsets[i] - current_offset};
const StringRef ref{&data[current_offset], offsets[i] - current_offset - 1};
current_offset = offsets[i];
const auto * it = table.find(ref);
if (it)
@ -593,15 +648,18 @@ namespace
{
using NumToIdx = HashMap<UInt64, size_t, HashCRC32<UInt64>>;
using StringToIdx = HashMap<StringRef, size_t, StringRefHash>;
using AnythingToIdx = HashMap<UInt128, size_t>;
std::unique_ptr<NumToIdx> table_num_to_idx;
std::unique_ptr<StringToIdx> table_string_to_idx;
std::unique_ptr<AnythingToIdx> table_anything_to_idx;
ColumnPtr to_columns;
bool is_empty = false;
ColumnPtr from_column;
ColumnPtr to_column;
ColumnPtr default_column;
Arena string_pool;
std::atomic<bool> initialized{false};
std::mutex mutex;
};
@ -609,27 +667,6 @@ namespace
mutable Cache cache;
static UInt64 bitCastToUInt64(const Field & x)
{
switch (x.getType())
{
case Field::Types::UInt64:
return x.get<UInt64>();
case Field::Types::Int64:
return x.get<Int64>();
case Field::Types::Float64:
return std::bit_cast<UInt64>(x.get<Float64>());
case Field::Types::Bool:
return x.get<bool>();
case Field::Types::Decimal32:
return x.get<DecimalField<Decimal32>>().getValue();
case Field::Types::Decimal64:
return x.get<DecimalField<Decimal64>>().getValue();
default:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unexpected type in function 'transform'");
}
}
static void checkAllowedType(const DataTypePtr & type)
{
if (type->isNullable())
@ -656,33 +693,56 @@ namespace
/// Can be called from different threads. It works only on the first call.
void initialize(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type) const
{
const ColumnConst * array_from = checkAndGetColumnConst<ColumnArray>(arguments[1].column.get());
const ColumnConst * array_to = checkAndGetColumnConst<ColumnArray>(arguments[2].column.get());
if (cache.initialized)
return;
const DataTypePtr & from_type = arguments[0].type;
if (from_type->onlyNull())
{
cache.is_empty = true;
return;
}
const ColumnArray * array_from = checkAndGetColumnConstData<ColumnArray>(arguments[1].column.get());
const ColumnArray * array_to = checkAndGetColumnConstData<ColumnArray>(arguments[2].column.get());
if (!array_from || !array_to)
throw Exception(
ErrorCodes::ILLEGAL_COLUMN, "Second and third arguments of function {} must be constant arrays.", getName());
if (cache.initialized)
return;
const auto & from = array_from->getValue<Array>();
const size_t size = from.size();
if (0 == size)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Empty arrays are illegal in function {}", getName());
std::lock_guard lock(cache.mutex);
if (cache.initialized)
return;
const ColumnPtr & from_column_uncasted = array_from->getDataPtr();
const auto & to = array_to->getValue<Array>();
if (size != to.size())
cache.from_column = castColumn(
{
from_column_uncasted,
typeid_cast<const DataTypeArray &>(*arguments[1].type).getNestedType(),
arguments[1].name
},
from_type);
cache.to_column = castColumn(
{
array_to->getDataPtr(),
typeid_cast<const DataTypeArray &>(*arguments[2].type).getNestedType(),
arguments[2].name
},
result_type);
const size_t size = cache.from_column->size();
if (0 == size)
{
cache.is_empty = true;
return;
}
if (cache.to_column->size() != size)
throw Exception(
ErrorCodes::BAD_ARGUMENTS, "Second and third arguments of function {} must be arrays of same size", getName());
/// Whether the default value is set.
if (arguments.size() == 4)
{
const IColumn * default_col = arguments[3].column.get();
@ -702,45 +762,54 @@ namespace
/// Note: Doesn't check the duplicates in the `from` array.
const IDataType & from_type = *arguments[0].type;
if (from[0].getType() != Field::Types::String)
WhichDataType which(from_type);
if (isNativeNumber(which) || which.isDecimal32() || which.isDecimal64())
{
cache.table_num_to_idx = std::make_unique<Cache::NumToIdx>();
auto & table = *cache.table_num_to_idx;
for (size_t i = 0; i < size; ++i)
{
Field key = convertFieldToType(from[i], from_type);
if (key.isNull())
continue;
/// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64
table[bitCastToUInt64(key)] = i;
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
{
/// Field may be of Float type, but for the purpose of bitwise equality we can treat them as UInt64
StringRef ref = cache.from_column->getDataAt(i);
UInt64 key = 0;
memcpy(&key, ref.data, ref.size);
table[key] = i;
}
}
}
else
else if (from_type->isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
{
cache.table_string_to_idx = std::make_unique<Cache::StringToIdx>();
auto & table = *cache.table_string_to_idx;
for (size_t i = 0; i < size; ++i)
{
const String & str_from = from[i].get<const String &>();
StringRef ref{cache.string_pool.insert(str_from.data(), str_from.size() + 1), str_from.size() + 1};
table[ref] = i;
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
{
StringRef ref = cache.from_column->getDataAt(i);
table[ref] = i;
}
}
}
else
{
cache.table_anything_to_idx = std::make_unique<Cache::AnythingToIdx>();
auto & table = *cache.table_anything_to_idx;
for (size_t i = 0; i < size; ++i)
{
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
{
SipHash hash;
cache.from_column->updateHashWithValue(i, hash);
table[hash.get128()] = i;
}
}
}
auto to_columns = result_type->createColumn();
for (size_t i = 0; i < size; ++i)
{
Field to_value = convertFieldToType(to[i], *result_type);
to_columns->insert(to_value);
}
cache.to_columns = std::move(to_columns);
cache.initialized = true;
}
};
};
}

View File

@ -57,6 +57,9 @@
#include <Functions/UserDefined/UserDefinedExecutableFunctionFactory.h>
#include <Parsers/QueryParameterVisitor.h>
#include <Analyzer/QueryNode.h>
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
#include <Parsers/queryToString.h>
namespace DB
{
@ -422,9 +425,8 @@ Block createBlockForSet(
}
SetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set,
ContextPtr context, const SizeLimits & size_limits, PreparedSets & prepared_sets)
FutureSetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets)
{
const IAST & args = *node->arguments;
@ -443,14 +445,16 @@ SetPtr makeExplicitSet(
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
set_element_types = left_tuple_type->getElements();
auto set_element_keys = Set::getElementTypes(set_element_types, context->getSettingsRef().transform_null_in);
auto set_key = right_arg->getTreeHash();
if (auto set = prepared_sets.findTuple(set_key, set_element_keys))
return set; /// Already prepared.
for (auto & element_type : set_element_types)
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(element_type.get()))
element_type = low_cardinality_type->getDictionaryType();
auto set_key = PreparedSetKey::forLiteral(*right_arg, set_element_types);
if (auto set = prepared_sets.get(set_key))
return set; /// Already prepared.
Block block;
const auto & right_arg_func = std::dynamic_pointer_cast<ASTFunction>(right_arg);
if (right_arg_func && (right_arg_func->name == "tuple" || right_arg_func->name == "array"))
@ -458,14 +462,7 @@ SetPtr makeExplicitSet(
else
block = createBlockForSet(left_arg_type, right_arg, set_element_types, context);
SetPtr set
= std::make_shared<Set>(size_limits, create_ordered_set, context->getSettingsRef().transform_null_in);
set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName());
set->insertFromBlock(block.getColumnsWithTypeAndName());
set->finishInsert();
prepared_sets.set(set_key, set);
return set;
return prepared_sets.addFromTuple(set_key, block, context->getSettings());
}
class ScopeStack::Index
@ -950,7 +947,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
return;
}
FutureSet prepared_set;
FutureSetPtr prepared_set;
if (checkFunctionIsInOrGlobalInOperator(node))
{
/// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything).
@ -959,7 +956,7 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
if (!data.no_makeset && !(data.is_create_parameterized_view && !analyzeReceiveQueryParams(ast).empty()))
prepared_set = makeSet(node, data, data.no_subqueries);
if (prepared_set.isValid())
if (prepared_set)
{
/// Transform tuple or subquery into a set.
}
@ -1171,14 +1168,14 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data &
num_arguments += columns.size() - 1;
arg += columns.size() - 1;
}
else if (checkFunctionIsInOrGlobalInOperator(node) && arg == 1 && prepared_set.isValid())
else if (checkFunctionIsInOrGlobalInOperator(node) && 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).
const bool is_constant_set = prepared_set.isCreated();
const bool is_constant_set = typeid_cast<const FutureSetFromSubquery *>(prepared_set.get()) == nullptr;
if (is_constant_set)
column.name = data.getUniqueName("__set");
else
@ -1381,7 +1378,7 @@ void ActionsMatcher::visit(const ASTLiteral & literal, const ASTPtr & /* ast */,
data.addColumn(std::move(column));
}
FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries)
FutureSetPtr ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no_subqueries)
{
if (!data.prepared_sets)
return {};
@ -1400,13 +1397,34 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
{
if (no_subqueries)
return {};
auto set_key = PreparedSetKey::forSubquery(*right_in_operand);
PreparedSets::Hash set_key;
if (data.getContext()->getSettingsRef().allow_experimental_analyzer && !identifier)
{
auto set = data.prepared_sets->getFuture(set_key);
if (set.isValid())
return set;
/// Here we can be only from mutation interpreter. Normal selects with analyzed use other interpreter.
/// This is a hacky way to allow reusing cache for prepared sets.
///
/// Mutation is executed in two stages:
/// * first, query 'SELECT count() FROM table WHERE ...' is executed to get the set of affected parts (using analyzer)
/// * second, every part is mutated separately, where plan is build "manually", using this code as well
/// To share the Set in between first and second stage, we should use the same hash.
/// New analyzer is uses a hash from query tree, so here we also build a query tree.
///
/// Note : this code can be safely removed, but the test 02581_share_big_sets will be too slow (and fail by timeout).
/// Note : we should use new analyzer for mutations and remove this hack.
InterpreterSelectQueryAnalyzer interpreter(right_in_operand, data.getContext(), SelectQueryOptions().analyze(true).subquery());
const auto & query_tree = interpreter.getQueryTree();
if (auto * query_node = query_tree->as<QueryNode>())
query_node->setIsSubquery(true);
set_key = query_tree->getTreeHash();
}
else
set_key = right_in_operand->getTreeHash();
if (auto set = data.prepared_sets->findSubquery(set_key))
return set;
FutureSetPtr external_table_set;
/// 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).
@ -1417,20 +1435,22 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
if (table)
{
StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get());
if (storage_set)
{
SetPtr set = storage_set->getSet();
data.prepared_sets->set(set_key, set);
return FutureSet(set);
}
if (StorageSet * storage_set = dynamic_cast<StorageSet *>(table.get()))
return data.prepared_sets->addFromStorage(set_key, storage_set->getSet());
}
if (!data.getContext()->isGlobalContext())
{
/// If we are reading from storage, it can be an external table which is used for GLOBAL IN.
/// Here, we take FutureSet which is used to build external table.
/// It will be used if set is useful for primary key. During PK analysis
/// temporary table is not filled yet, so we need to fill it first.
if (auto tmp_table = data.getContext()->findExternalTable(identifier->getColumnName()))
external_table_set = tmp_table->future_set;
}
}
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
String set_id = right_in_operand->getColumnName();
bool transform_null_in = data.getContext()->getSettingsRef().transform_null_in;
SubqueryForSet & subquery_for_set = data.prepared_sets->createOrGetSubquery(set_id, set_key, data.set_size_limit, transform_null_in);
std::unique_ptr<QueryPlan> source = std::make_unique<QueryPlan>();
/** The following happens for GLOBAL INs or INs:
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
@ -1440,13 +1460,12 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
* In case that we have HAVING with IN subquery, we have to force creating set for it.
* Also it doesn't make sense if it is GLOBAL IN or ordinary IN.
*/
if (!subquery_for_set.hasSource())
{
auto interpreter = interpretSubquery(right_in_operand, data.getContext(), data.subquery_depth, {});
subquery_for_set.createSource(*interpreter);
interpreter->buildQueryPlan(*source);
}
return subquery_for_set.set;
return data.prepared_sets->addFromSubquery(set_key, std::move(source), nullptr, std::move(external_table_set), data.getContext()->getSettingsRef());
}
else
{
@ -1454,8 +1473,7 @@ FutureSet ActionsMatcher::makeSet(const ASTFunction & node, Data & data, bool no
const auto & index = data.actions_stack.getLastActionsIndex();
if (data.prepared_sets && index.contains(left_in_operand->getColumnName()))
/// An explicit enumeration of values in parentheses.
return FutureSet(
makeExplicitSet(&node, last_actions, false, data.getContext(), data.set_size_limit, *data.prepared_sets));
return makeExplicitSet(&node, last_actions, data.getContext(), *data.prepared_sets);
else
return {};
}

View File

@ -25,9 +25,8 @@ class IFunctionOverloadResolver;
using FunctionOverloadResolverPtr = std::shared_ptr<IFunctionOverloadResolver>;
/// The case of an explicit enumeration of values.
SetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG & actions, bool create_ordered_set,
ContextPtr context, const SizeLimits & limits, PreparedSets & prepared_sets);
FutureSetPtr makeExplicitSet(
const ASTFunction * node, const ActionsDAG & actions, ContextPtr context, PreparedSets & prepared_sets);
/** For ActionsVisitor
* A stack of ExpressionActions corresponding to nested lambda expressions.
@ -217,7 +216,7 @@ private:
static void visit(const ASTLiteral & literal, const ASTPtr & ast, Data & data);
static void visit(ASTExpressionList & expression_list, const ASTPtr & ast, Data & data);
static FutureSet makeSet(const ASTFunction & node, Data & data, bool no_subqueries);
static FutureSetPtr makeSet(const ASTFunction & node, Data & data, bool no_subqueries);
static ASTs doUntuple(const ASTFunction * function, ActionsMatcher::Data & data);
static std::optional<NameAndTypePair> getNameAndTypeFromAST(const ASTPtr & ast, Data & data);
};

View File

@ -498,12 +498,14 @@ KeyMetadata::iterator FileCache::addFileSegment(
chassert(size > 0); /// Empty file segments in cache are not allowed.
const auto & key = locked_key.getKey();
if (locked_key.tryGetByOffset(offset))
const FileSegment::Range range(offset, offset + size - 1);
if (auto intersecting_range = locked_key.hasIntersectingRange(range))
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cache entry already exists for key: `{}`, offset: {}, size: {}.",
key, offset, size);
"Attempt to add intersecting file segment in cache ({} intersects {})",
range.toString(), intersecting_range->toString());
}
FileSegment::State result_state;
@ -958,8 +960,20 @@ void FileCache::loadMetadata()
if ((main_priority->getSizeLimit() == 0 || main_priority->getSize(lock) + size <= main_priority->getSizeLimit())
&& (main_priority->getElementsLimit() == 0 || main_priority->getElementsCount(lock) + 1 <= main_priority->getElementsLimit()))
{
auto file_segment_metadata_it = addFileSegment(
*locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock);
KeyMetadata::iterator file_segment_metadata_it;
try
{
file_segment_metadata_it = addFileSegment(
*locked_key, offset, size, FileSegment::State::DOWNLOADED, CreateFileSegmentSettings(segment_kind), &lock);
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
chassert(false);
fs::remove(offset_it->path());
continue;
}
const auto & file_segment_metadata = file_segment_metadata_it->second;
chassert(file_segment_metadata->file_segment->assertCorrectness());

View File

@ -129,6 +129,8 @@ public:
bool operator==(const Range & other) const { return left == other.left && right == other.right; }
bool operator<(const Range & other) const { return right < other.left; }
size_t size() const { return right - left + 1; }
String toString() const { return fmt::format("[{}, {}]", std::to_string(left), std::to_string(right)); }

View File

@ -448,6 +448,29 @@ void LockedKey::shrinkFileSegmentToDownloadedSize(
chassert(file_segment->assertCorrectnessUnlocked(segment_lock));
}
std::optional<FileSegment::Range> LockedKey::hasIntersectingRange(const FileSegment::Range & range) const
{
if (key_metadata->empty())
return {};
auto it = key_metadata->lower_bound(range.left);
if (it != key_metadata->end()) /// has next range
{
auto next_range = it->second->file_segment->range();
if (!(range < next_range))
return next_range;
if (it == key_metadata->begin())
return {};
}
auto prev_range = std::prev(it)->second->file_segment->range();
if (!(prev_range < range))
return prev_range;
return {};
}
std::shared_ptr<const FileSegmentMetadata> LockedKey::getByOffset(size_t offset) const
{
auto it = key_metadata->find(offset);

View File

@ -164,6 +164,8 @@ struct LockedKey : private boost::noncopyable
bool isLastOwnerOfFileSegment(size_t offset) const;
std::optional<FileSegment::Range> hasIntersectingRange(const FileSegment::Range & range) const;
void removeFromCleanupQueue();
void markAsRemoved();

View File

@ -132,6 +132,11 @@ QueryCache::Key::Key(
{
}
QueryCache::Key::Key(ASTPtr ast_, const String & user_name_)
: QueryCache::Key(ast_, {}, user_name_, false, std::chrono::system_clock::from_time_t(1), false) /// dummy values for everything != AST or user name
{
}
bool QueryCache::Key::operator==(const Key & other) const
{
return ast->getTreeHash() == other.ast->getTreeHash();
@ -387,19 +392,22 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
return;
}
if (!entry->key.is_shared && entry->key.user_name != key.user_name)
const auto & entry_key = entry->key;
const auto & entry_mapped = entry->mapped;
if (!entry_key.is_shared && entry_key.user_name != key.user_name)
{
LOG_TRACE(&Poco::Logger::get("QueryCache"), "Inaccessible entry found for query {}", key.queryStringFromAst());
return;
}
if (IsStale()(entry->key))
if (IsStale()(entry_key))
{
LOG_TRACE(&Poco::Logger::get("QueryCache"), "Stale entry found for query {}", key.queryStringFromAst());
return;
}
if (!entry->key.is_compressed)
if (!entry_key.is_compressed)
{
// Cloning chunks isn't exactly great. It could be avoided by another indirection, i.e. wrapping Entry's members chunks, totals and
// extremes into shared_ptrs and assuming that the lifecycle of these shared_ptrs coincides with the lifecycle of the Entry
@ -408,15 +416,15 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
// optimization.
Chunks cloned_chunks;
for (const auto & chunk : entry->mapped->chunks)
for (const auto & chunk : entry_mapped->chunks)
cloned_chunks.push_back(chunk.clone());
buildSourceFromChunks(entry->key.header, std::move(cloned_chunks), entry->mapped->totals, entry->mapped->extremes);
buildSourceFromChunks(entry_key.header, std::move(cloned_chunks), entry_mapped->totals, entry_mapped->extremes);
}
else
{
Chunks decompressed_chunks;
const Chunks & chunks = entry->mapped->chunks;
const Chunks & chunks = entry_mapped->chunks;
for (const auto & chunk : chunks)
{
const Columns & columns = chunk.getColumns();
@ -430,7 +438,7 @@ QueryCache::Reader::Reader(Cache & cache_, const Key & key, const std::lock_guar
decompressed_chunks.push_back(std::move(decompressed_chunk));
}
buildSourceFromChunks(entry->key.header, std::move(decompressed_chunks), entry->mapped->totals, entry->mapped->extremes);
buildSourceFromChunks(entry_key.header, std::move(decompressed_chunks), entry_mapped->totals, entry_mapped->extremes);
}
LOG_TRACE(&Poco::Logger::get("QueryCache"), "Entry found for query {}", key.queryStringFromAst());

View File

@ -49,7 +49,7 @@ public:
/// If the associated entry can be read by other users. In general, sharing is a bad idea: First, it is unlikely that different
/// users pose the same queries. Second, sharing potentially breaches security. E.g. User A should not be able to bypass row
/// policies on some table by running the same queries as user B for whom no row policies exist.
bool is_shared;
const bool is_shared;
/// When does the entry expire?
const std::chrono::time_point<std::chrono::system_clock> expires_at;
@ -58,12 +58,16 @@ public:
/// (we could theoretically apply compression also to the totals and extremes but it's an obscure use case)
const bool is_compressed;
/// Ctor to construct a Key for writing into query cache.
Key(ASTPtr ast_,
Block header_,
const String & user_name_, bool is_shared_,
std::chrono::time_point<std::chrono::system_clock> expires_at_,
bool is_compressed);
/// Ctor to construct a Key for reading from query cache (this operation only needs the AST + user name).
Key(ASTPtr ast_, const String & user_name_);
bool operator==(const Key & other) const;
String queryStringFromAst() const;
};

View File

@ -26,6 +26,8 @@ using QueryPlanPtr = std::unique_ptr<QueryPlan>;
struct StorageID;
class PreparedSets;
using PreparedSetsPtr = std::shared_ptr<PreparedSets>;
namespace ClusterProxy
{

View File

@ -18,6 +18,7 @@
#include <Common/Exception.h>
#include <Common/WeakHash.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace DB
{

View File

@ -1319,6 +1319,21 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder &
external_tables_mapping.emplace(table_name, std::make_shared<TemporaryTableHolder>(std::move(temporary_table)));
}
std::shared_ptr<TemporaryTableHolder> Context::findExternalTable(const String & table_name) const
{
if (isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot have external tables");
std::shared_ptr<TemporaryTableHolder> holder;
{
auto lock = getLock();
auto iter = external_tables_mapping.find(table_name);
if (iter == external_tables_mapping.end())
return {};
holder = iter->second;
}
return holder;
}
std::shared_ptr<TemporaryTableHolder> Context::removeExternalTable(const String & table_name)
{

View File

@ -615,6 +615,7 @@ public:
Tables getExternalTables() const;
void addExternalTable(const String & table_name, TemporaryTableHolder && temporary_table);
std::shared_ptr<TemporaryTableHolder> findExternalTable(const String & table_name) const;
std::shared_ptr<TemporaryTableHolder> removeExternalTable(const String & table_name);
const Scalars & getScalars() const;

View File

@ -110,7 +110,7 @@ TemporaryTableHolder::TemporaryTableHolder(
}
TemporaryTableHolder::TemporaryTableHolder(TemporaryTableHolder && rhs) noexcept
: WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id)
: WithContext(rhs.context), temporary_tables(rhs.temporary_tables), id(rhs.id), future_set(std::move(rhs.future_set))
{
rhs.id = UUIDHelpers::Nil;
}

View File

@ -79,6 +79,8 @@ private:
using DDLGuardPtr = std::unique_ptr<DDLGuard>;
class FutureSet;
using FutureSetPtr = std::shared_ptr<FutureSet>;
/// Creates temporary table in `_temporary_and_external_tables` with randomly generated unique StorageID.
/// Such table can be accessed from everywhere by its ID.
@ -111,6 +113,7 @@ struct TemporaryTableHolder : boost::noncopyable, WithContext
IDatabase * temporary_tables = nullptr;
UUID id = UUIDHelpers::Nil;
FutureSetPtr future_set;
};
///TODO maybe remove shared_ptr from here?

View File

@ -936,15 +936,12 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con
for (const auto & action : actions)
{
if (action.node->type == ActionsDAG::ActionType::COLUMN && action.node->result_name == set_to_check)
{
// Constant ColumnSet cannot be empty, so we only need to check non-constant ones.
if (const auto * column_set = checkAndGetColumn<const ColumnSet>(action.node->column.get()))
{
auto set = column_set->getData();
if (set && set->isCreated() && set->getTotalRowCount() == 0)
return true;
}
}
if (auto future_set = column_set->getData())
if (auto set = future_set->get())
if (set->getTotalRowCount() == 0)
return true;
}
}

View File

@ -450,77 +450,6 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global, b
}
void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options)
{
if (!prepared_sets)
return;
auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name);
if (prepared_sets->getFuture(set_key).isValid())
return; /// Already prepared.
if (auto set_ptr_from_storage_set = isPlainStorageSetInSubquery(subquery_or_table_name))
{
prepared_sets->set(set_key, set_ptr_from_storage_set);
return;
}
auto build_set = [&] () -> SetPtr
{
LOG_TRACE(getLogger(), "Building set, key: {}", set_key.toString());
auto interpreter_subquery = interpretSubquery(subquery_or_table_name, getContext(), {}, query_options);
auto io = interpreter_subquery->execute();
PullingAsyncPipelineExecutor executor(io.pipeline);
SetPtr set = std::make_shared<Set>(settings.size_limits_for_set_used_with_index, true, getContext()->getSettingsRef().transform_null_in);
set->setHeader(executor.getHeader().getColumnsWithTypeAndName());
Block block;
while (executor.pull(block))
{
if (block.rows() == 0)
continue;
/// If the limits have been exceeded, give up and let the default subquery processing actions take place.
if (!set->insertFromBlock(block.getColumnsWithTypeAndName()))
return nullptr;
}
set->finishInsert();
return set;
};
SetPtr set;
auto set_cache = getContext()->getPreparedSetsCache();
if (set_cache)
{
auto from_cache = set_cache->findOrPromiseToBuild(set_key.toString());
if (from_cache.index() == 0)
{
set = build_set();
std::get<0>(from_cache).set_value(set);
}
else
{
LOG_TRACE(getLogger(), "Waiting for set, key: {}", set_key.toString());
set = std::get<1>(from_cache).get();
}
}
else
{
set = build_set();
}
if (!set)
return;
prepared_sets->set(set_key, std::move(set));
}
SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name)
{
const auto * table = subquery_or_table_name->as<ASTTableIdentifier>();
@ -534,54 +463,6 @@ SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_o
return storage_set->getSet();
}
/// Performance optimization 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.
if (child->as<ASTSubquery>())
continue;
/// Don't descend into lambda functions
const auto * func = child->as<ASTFunction>();
if (func && func->name == "lambda")
continue;
makeSetsForIndex(child);
}
const auto * func = node->as<ASTFunction>();
if (func && functionIsInOrGlobalInOperator(func->name))
{
const IAST & args = *func->arguments;
const ASTPtr & left_in_operand = args.children.at(0);
if (storage()->mayBenefitFromIndexForIn(left_in_operand, getContext(), metadata_snapshot))
{
const ASTPtr & arg = args.children.at(1);
if (arg->as<ASTSubquery>() || arg->as<ASTTableIdentifier>())
{
if (settings.use_index_for_in_with_subqueries)
tryMakeSetForIndexFromSubquery(arg, query_options);
}
else
{
auto temp_actions = std::make_shared<ActionsDAG>(columns_after_join);
getRootActions(left_in_operand, true, temp_actions);
if (prepared_sets && temp_actions->tryFindInOutputs(left_in_operand->getColumnName()))
makeExplicitSet(func, *temp_actions, true, getContext(), settings.size_limits_for_set, *prepared_sets);
}
}
}
}
void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_makeset_for_subqueries, ActionsDAGPtr & actions, bool only_consts)
{
LogAST log;

View File

@ -141,11 +141,6 @@ public:
void makeWindowDescriptionFromAST(const Context & context, const WindowDescriptions & existing_descriptions, WindowDescription & desc, const IAST * ast);
void makeWindowDescriptions(ActionsDAGPtr actions);
/** Create Set from a subquery or a table expression in the query. The created set is suitable for using the index.
* The set will not be created if its size hits the limit.
*/
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {});
/** Checks if subquery is not a plain StorageSet.
* Because while making set we will read data from StorageSet which is not allowed.
* Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise.
@ -363,9 +358,6 @@ public:
/// Deletes all columns except mentioned by SELECT, arranges the remaining columns and renames them to aliases.
ActionsDAGPtr appendProjectResult(ExpressionActionsChain & chain) const;
/// Create Set-s that we make from IN section to use index on them.
void makeSetsForIndex(const ASTPtr & node);
private:
StorageMetadataPtr metadata_snapshot;
/// If non-empty, ignore all expressions not from this list.

View File

@ -31,6 +31,7 @@ namespace DB
namespace ErrorCodes
{
extern const int WRONG_GLOBAL_SUBQUERY;
extern const int LOGICAL_ERROR;
}
class GlobalSubqueriesMatcher
@ -161,30 +162,20 @@ public:
nullptr,
/*create_for_global_subquery*/ true);
StoragePtr external_storage = external_storage_holder->getTable();
external_tables.emplace(external_table_name, external_storage_holder);
/// We need to materialize external tables immediately because reading from distributed
/// tables might generate local plans which can refer to external tables during index
/// analysis. It's too late to populate the external table via CreatingSetsTransform.
if (is_explain)
auto set_key = database_and_table_name->getTreeHash();
if (!prepared_sets->findSubquery(set_key))
{
/// Do not materialize external tables if it's explain statement.
}
else if (getContext()->getSettingsRef().use_index_for_in_with_subqueries)
{
auto external_table = external_storage_holder->getTable();
auto table_out = external_table->write({}, external_table->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false);
auto io = interpreter->execute();
io.pipeline.complete(std::move(table_out));
CompletedPipelineExecutor executor(io.pipeline);
executor.execute();
std::unique_ptr<QueryPlan> source = std::make_unique<QueryPlan>();
interpreter->buildQueryPlan(*source);
auto future_set = prepared_sets->addFromSubquery(set_key, std::move(source), std::move(external_storage), nullptr, getContext()->getSettingsRef());
external_storage_holder->future_set = std::move(future_set);
}
else
{
auto & subquery_for_set = prepared_sets->getSubquery(external_table_name);
subquery_for_set.createSource(*interpreter, external_storage);
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Set is already created for GLOBAL IN");
/** NOTE If it was written IN tmp_table - the existing temporary (but not external) table,
* then a new temporary table will be created (for example, _data1),

View File

@ -953,10 +953,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
if (storage && !options.only_analyze)
{
query_analyzer->makeSetsForIndex(select_query.where());
query_analyzer->makeSetsForIndex(select_query.prewhere());
query_info.prepared_sets = query_analyzer->getPreparedSets();
from_stage = storage->getQueryProcessingStage(context, options.to_stage, storage_snapshot, query_info);
}
@ -3151,7 +3148,17 @@ void InterpreterSelectQuery::executeExtremes(QueryPlan & query_plan)
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPlan & query_plan)
{
addCreatingSetsStep(query_plan, prepared_sets, context);
auto subqueries = prepared_sets->getSubqueries();
if (!subqueries.empty())
{
auto step = std::make_unique<DelayedCreatingSetsStep>(
query_plan.getCurrentDataStream(),
std::move(subqueries),
context);
query_plan.addStep(std::move(step));
}
}

View File

@ -69,6 +69,8 @@ public:
const Planner & getPlanner() const { return planner; }
Planner & getPlanner() { return planner; }
const QueryTreeNodePtr & getQueryTree() const { return query_tree; }
private:
ASTPtr query;
ContextMutablePtr context;

View File

@ -326,6 +326,8 @@ void MergeTreeTransaction::afterFinalize()
is_read_only = storages.empty();
/// Release shared pointers just in case
creating_parts.clear();
removing_parts.clear();
storages.clear();
mutations.clear();
finalized = true;

View File

@ -1,199 +0,0 @@
#include <Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h>
#include <Core/Field.h>
#include <Core/NamesAndTypes.h>
#include <Common/DateLUT.h>
#include <Common/DateLUTImpl.h>
#include <Functions/FunctionFactory.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** Given a monotonic non-decreasing function f(x), which satisfies f(x) = c for any value x within [b, e).
* We could convert it into its equivalent form, x >= b AND x < e, which is free from the invocation of the function.
* And we could apply the similar transformation to other comparisons. The suggested transformations list:
*
* f(x) == c -> x >= b AND x < e
* f(x) != c -> x < b OR x >= e
* f(x) > c -> x >= e
* f(x) >= c -> x >= b
* f(x) < c -> x < b
* f(x) <= c -> x < e
*
* This function generates a new AST with the transformed relation.
*/
ASTPtr generateOptimizedDateFilterAST(const String & comparator, const NameAndTypePair & column, const std::pair<Field, Field>& range)
{
const DateLUTImpl & date_lut = DateLUT::instance();
const String & column_name = column.name;
String start_date_or_date_time;
String end_date_or_date_time;
if (isDateOrDate32(column.type.get()))
{
start_date_or_date_time = date_lut.dateToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.dateToString(range.second.get<DateLUTImpl::Time>());
}
else if (isDateTime(column.type.get()) || isDateTime64(column.type.get()))
{
start_date_or_date_time = date_lut.timeToString(range.first.get<DateLUTImpl::Time>());
end_date_or_date_time = date_lut.timeToString(range.second.get<DateLUTImpl::Time>());
}
else [[unlikely]] return {};
if (comparator == "equals")
{
return makeASTFunction("and",
makeASTFunction("greaterOrEquals",
std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(start_date_or_date_time)
),
makeASTFunction("less",
std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(end_date_or_date_time)
)
);
}
else if (comparator == "notEquals")
{
return makeASTFunction("or",
makeASTFunction("less",
std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(start_date_or_date_time)
),
makeASTFunction("greaterOrEquals",
std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(end_date_or_date_time)
)
);
}
else if (comparator == "greater")
{
return makeASTFunction("greaterOrEquals",
std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(end_date_or_date_time)
);
}
else if (comparator == "lessOrEquals")
{
return makeASTFunction("less",
std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(end_date_or_date_time)
);
}
else if (comparator == "less" || comparator == "greaterOrEquals")
{
return makeASTFunction(comparator,
std::make_shared<ASTIdentifier>(column_name),
std::make_shared<ASTLiteral>(start_date_or_date_time)
);
}
else [[unlikely]]
{
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Expected equals, notEquals, less, lessOrEquals, greater, greaterOrEquals. Actual {}",
comparator);
}
}
void OptimizeDateOrDateTimeConverterWithPreimageMatcher::visit(const ASTFunction & function, ASTPtr & ast, const Data & data)
{
const static std::unordered_map<String, String> swap_relations = {
{"equals", "equals"},
{"notEquals", "notEquals"},
{"less", "greater"},
{"greater", "less"},
{"lessOrEquals", "greaterOrEquals"},
{"greaterOrEquals", "lessOrEquals"},
};
if (!swap_relations.contains(function.name)) return;
if (!function.arguments || function.arguments->children.size() != 2) return;
size_t func_id = function.arguments->children.size();
for (size_t i = 0; i < function.arguments->children.size(); i++)
{
if (const auto * func = function.arguments->children[i]->as<ASTFunction>())
{
func_id = i;
}
}
if (func_id == function.arguments->children.size()) return;
size_t literal_id = 1 - func_id;
const auto * literal = function.arguments->children[literal_id]->as<ASTLiteral>();
if (!literal || literal->value.getType() != Field::Types::UInt64) return;
String comparator = literal_id > func_id ? function.name : swap_relations.at(function.name);
const auto * ast_func = function.arguments->children[func_id]->as<ASTFunction>();
/// Currently we only handle single-argument functions.
if (!ast_func || !ast_func->arguments || ast_func->arguments->children.size() != 1) return;
const auto * column_id = ast_func->arguments->children.at(0)->as<ASTIdentifier>();
if (!column_id) return;
auto pos = IdentifierSemantic::getMembership(*column_id);
if (!pos)
pos = IdentifierSemantic::chooseTableColumnMatch(*column_id, data.tables, true);
if (!pos)
return;
if (*pos >= data.tables.size())
return;
auto data_type_and_name = data.tables[*pos].columns.tryGetByName(column_id->shortName());
if (!data_type_and_name) return;
const auto & converter = FunctionFactory::instance().tryGet(ast_func->name, data.context);
if (!converter) return;
ColumnsWithTypeAndName args;
args.emplace_back(data_type_and_name->type, "tmp");
auto converter_base = converter->build(args);
if (!converter_base || !converter_base->hasInformationAboutPreimage()) return;
auto preimage_range = converter_base->getPreimage(*(data_type_and_name->type), literal->value);
if (!preimage_range) return;
const auto new_ast = generateOptimizedDateFilterAST(comparator, *data_type_and_name, *preimage_range);
if (!new_ast) return;
ast = new_ast;
}
bool OptimizeDateOrDateTimeConverterWithPreimageMatcher::needChildVisit(ASTPtr & ast, ASTPtr & /*child*/)
{
const static std::unordered_set<String> relations = {
"equals",
"notEquals",
"less",
"greater",
"lessOrEquals",
"greaterOrEquals",
};
if (const auto * ast_function = ast->as<ASTFunction>())
{
return !relations.contains(ast_function->name);
}
return true;
}
}

View File

@ -1,37 +0,0 @@
#pragma once
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/InDepthNodeVisitor.h>
namespace DB
{
class ASTFunction;
/** Replace predicate having Date/DateTime converters with their preimages to improve performance.
* Given a Date column c, toYear(c) = 2023 -> c >= '2023-01-01' AND c < '2024-01-01'
* Or if c is a DateTime column, toYear(c) = 2023 -> c >= '2023-01-01 00:00:00' AND c < '2024-01-01 00:00:00'.
* The similar optimization also applies to other converters.
*/
class OptimizeDateOrDateTimeConverterWithPreimageMatcher
{
public:
struct Data
{
const TablesWithColumns & tables;
ContextPtr context;
};
static void visit(ASTPtr & ast, Data & data)
{
if (const auto * ast_function = ast->as<ASTFunction>())
visit(*ast_function, ast, data);
}
static void visit(const ASTFunction & function, ASTPtr & ast, const Data & data);
static bool needChildVisit(ASTPtr & ast, ASTPtr & child);
};
using OptimizeDateOrDateTimeConverterWithPreimageVisitor = InDepthNodeVisitor<OptimizeDateOrDateTimeConverterWithPreimageMatcher, true>;
}

View File

@ -1,55 +1,213 @@
#include <chrono>
#include <variant>
#include <Interpreters/PreparedSets.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/Set.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <IO/Operators.h>
#include <Common/logger_useful.h>
#include <Processors/QueryPlan/CreatingSetsStep.h>
#include <Processors/Executors/CompletedPipelineExecutor.h>
#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <Processors/Sinks/EmptySink.h>
#include <Processors/Sinks/NullSink.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Core/Block.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <QueryPipeline/SizeLimits.h>
namespace DB
{
PreparedSetKey PreparedSetKey::forLiteral(const IAST & ast, DataTypes types_)
namespace ErrorCodes
{
/// Remove LowCardinality types from type list because Set doesn't support LowCardinality keys now,
/// just converts LowCardinality to ordinary types.
for (auto & type : types_)
type = recursiveRemoveLowCardinality(type);
PreparedSetKey key;
key.ast_hash = ast.getTreeHash();
key.types = std::move(types_);
return key;
extern const int LOGICAL_ERROR;
}
PreparedSetKey PreparedSetKey::forSubquery(const IAST & ast)
static SizeLimits getSizeLimitsForSet(const Settings & settings)
{
PreparedSetKey key;
key.ast_hash = ast.getTreeHash();
return key;
return SizeLimits(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode);
}
bool PreparedSetKey::operator==(const PreparedSetKey & other) const
static bool equals(const DataTypes & lhs, const DataTypes & rhs)
{
if (ast_hash != other.ast_hash)
size_t size = lhs.size();
if (size != rhs.size())
return false;
if (types.size() != other.types.size())
return false;
for (size_t i = 0; i < types.size(); ++i)
for (size_t i = 0; i < size; ++i)
{
if (!types[i]->equals(*other.types[i]))
if (!lhs[i]->equals(*rhs[i]))
return false;
}
return true;
}
String PreparedSetKey::toString() const
FutureSetFromStorage::FutureSetFromStorage(SetPtr set_) : set(std::move(set_)) {}
SetPtr FutureSetFromStorage::get() const { return set; }
const DataTypes & FutureSetFromStorage::getTypes() const { return set->getElementsTypes(); }
SetPtr FutureSetFromStorage::buildOrderedSetInplace(const ContextPtr &)
{
return set->hasExplicitSetElements() ? set : nullptr;
}
FutureSetFromTuple::FutureSetFromTuple(Block block, const Settings & settings)
{
auto size_limits = getSizeLimitsForSet(settings);
set = std::make_shared<Set>(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in);
set->setHeader(block.cloneEmpty().getColumnsWithTypeAndName());
Columns columns;
columns.reserve(block.columns());
for (const auto & column : block)
columns.emplace_back(column.column);
set_key_columns.filter = ColumnUInt8::create(block.rows());
set->insertFromColumns(columns, set_key_columns);
set->finishInsert();
}
const DataTypes & FutureSetFromTuple::getTypes() const { return set->getElementsTypes(); }
SetPtr FutureSetFromTuple::buildOrderedSetInplace(const ContextPtr & context)
{
if (set->hasExplicitSetElements())
return set;
const auto & settings = context->getSettingsRef();
size_t max_values = settings.use_index_for_in_with_subqueries_max_values;
bool too_many_values = max_values && max_values < set->getTotalRowCount();
if (!too_many_values)
{
set->fillSetElements();
set->appendSetElements(set_key_columns);
}
return set;
}
FutureSetFromSubquery::FutureSetFromSubquery(
String key,
std::unique_ptr<QueryPlan> source_,
StoragePtr external_table_,
FutureSetPtr external_table_set_,
const Settings & settings)
: external_table(std::move(external_table_))
, external_table_set(std::move(external_table_set_))
, source(std::move(source_))
{
set_and_key = std::make_shared<SetAndKey>();
set_and_key->key = std::move(key);
auto size_limits = getSizeLimitsForSet(settings);
set_and_key->set = std::make_shared<Set>(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in);
set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName());
}
FutureSetFromSubquery::FutureSetFromSubquery(
String key,
QueryTreeNodePtr query_tree_,
const Settings & settings)
: query_tree(std::move(query_tree_))
{
set_and_key = std::make_shared<SetAndKey>();
set_and_key->key = std::move(key);
auto size_limits = getSizeLimitsForSet(settings);
set_and_key->set = std::make_shared<Set>(size_limits, settings.use_index_for_in_with_subqueries_max_values, settings.transform_null_in);
}
SetPtr FutureSetFromSubquery::get() const
{
if (set_and_key->set != nullptr && set_and_key->set->isCreated())
return set_and_key->set;
return nullptr;
}
void FutureSetFromSubquery::setQueryPlan(std::unique_ptr<QueryPlan> source_)
{
source = std::move(source_);
set_and_key->set->setHeader(source->getCurrentDataStream().header.getColumnsWithTypeAndName());
}
const DataTypes & FutureSetFromSubquery::getTypes() const
{
return set_and_key->set->getElementsTypes();
}
std::unique_ptr<QueryPlan> FutureSetFromSubquery::build(const ContextPtr & context)
{
if (set_and_key->set->isCreated())
return nullptr;
const auto & settings = context->getSettingsRef();
auto plan = std::move(source);
if (!plan)
return nullptr;
auto creating_set = std::make_unique<CreatingSetStep>(
plan->getCurrentDataStream(),
set_and_key,
external_table,
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode),
context);
creating_set->setStepDescription("Create set for subquery");
plan->addStep(std::move(creating_set));
return plan;
}
SetPtr FutureSetFromSubquery::buildOrderedSetInplace(const ContextPtr & context)
{
if (!context->getSettingsRef().use_index_for_in_with_subqueries)
return nullptr;
if (auto set = get())
{
if (set->hasExplicitSetElements())
return set;
return nullptr;
}
if (external_table_set)
{
auto set = external_table_set->buildOrderedSetInplace(context);
if (set)
return set_and_key->set = set;
}
auto plan = build(context);
if (!plan)
return nullptr;
set_and_key->set->fillSetElements();
auto builder = plan->buildQueryPipeline(QueryPlanOptimizationSettings::fromContext(context), BuildQueryPipelineSettings::fromContext(context));
auto pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder));
pipeline.complete(std::make_shared<EmptySink>(Block()));
CompletedPipelineExecutor executor(pipeline);
executor.execute();
set_and_key->set->checkIsCreated();
return set_and_key->set;
}
String PreparedSets::toString(const PreparedSets::Hash & key, const DataTypes & types)
{
WriteBufferFromOwnString buf;
buf << "__set_" << ast_hash.first << "_" << ast_hash.second;
buf << "__set_" << key.first << "_" << key.second;
if (!types.empty())
{
buf << "(";
@ -66,114 +224,112 @@ String PreparedSetKey::toString() const
return buf.str();
}
SubqueryForSet & PreparedSets::createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key,
SizeLimits set_size_limit, bool transform_null_in)
FutureSetPtr PreparedSets::addFromTuple(const Hash & key, Block block, const Settings & settings)
{
SubqueryForSet & subquery = subqueries[subquery_id];
auto from_tuple = std::make_shared<FutureSetFromTuple>(std::move(block), settings);
const auto & set_types = from_tuple->getTypes();
auto & sets_by_hash = sets_from_tuple[key];
/// If you already created a Set with the same subquery / table for another ast
/// In that case several PreparedSetKey would share same subquery and set
/// Not sure if it's really possible case (maybe for distributed query when set was filled by external table?)
if (subquery.set.isValid())
sets[key] = subquery.set;
else
{
subquery.set_in_progress = std::make_shared<Set>(set_size_limit, false, transform_null_in);
sets[key] = FutureSet(subquery.promise_to_fill_set.get_future());
}
for (const auto & set : sets_by_hash)
if (equals(set->getTypes(), set_types))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, set_types));
if (!subquery.set_in_progress)
{
subquery.key = key.toString();
subquery.set_in_progress = std::make_shared<Set>(set_size_limit, false, transform_null_in);
}
return subquery;
sets_by_hash.push_back(from_tuple);
return from_tuple;
}
/// If the subquery is not associated with any set, create default-constructed SubqueryForSet.
/// It's aimed to fill external table passed to SubqueryForSet::createSource.
SubqueryForSet & PreparedSets::getSubquery(const String & subquery_id) { return subqueries[subquery_id]; }
void PreparedSets::set(const PreparedSetKey & key, SetPtr set_) { sets[key] = FutureSet(set_); }
FutureSet PreparedSets::getFuture(const PreparedSetKey & key) const
FutureSetPtr PreparedSets::addFromStorage(const Hash & key, SetPtr set_)
{
auto it = sets.find(key);
if (it == sets.end())
return {};
auto from_storage = std::make_shared<FutureSetFromStorage>(std::move(set_));
auto [it, inserted] = sets_from_storage.emplace(key, from_storage);
if (!inserted)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {}));
return from_storage;
}
FutureSetPtr PreparedSets::addFromSubquery(
const Hash & key,
std::unique_ptr<QueryPlan> source,
StoragePtr external_table,
FutureSetPtr external_table_set,
const Settings & settings)
{
auto from_subquery = std::make_shared<FutureSetFromSubquery>(
toString(key, {}),
std::move(source),
std::move(external_table),
std::move(external_table_set),
settings);
auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery);
if (!inserted)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {}));
return from_subquery;
}
FutureSetPtr PreparedSets::addFromSubquery(
const Hash & key,
QueryTreeNodePtr query_tree,
const Settings & settings)
{
auto from_subquery = std::make_shared<FutureSetFromSubquery>(
toString(key, {}),
std::move(query_tree),
settings);
auto [it, inserted] = sets_from_subqueries.emplace(key, from_subquery);
if (!inserted)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Duplicate set: {}", toString(key, {}));
return from_subquery;
}
FutureSetPtr PreparedSets::findTuple(const Hash & key, const DataTypes & types) const
{
auto it = sets_from_tuple.find(key);
if (it == sets_from_tuple.end())
return nullptr;
for (const auto & set : it->second)
if (equals(set->getTypes(), types))
return set;
return nullptr;
}
std::shared_ptr<FutureSetFromSubquery> PreparedSets::findSubquery(const Hash & key) const
{
auto it = sets_from_subqueries.find(key);
if (it == sets_from_subqueries.end())
return nullptr;
return it->second;
}
SetPtr PreparedSets::get(const PreparedSetKey & key) const
std::shared_ptr<FutureSetFromStorage> PreparedSets::findStorage(const Hash & key) const
{
auto it = sets.find(key);
if (it == sets.end() || !it->second.isReady())
auto it = sets_from_storage.find(key);
if (it == sets_from_storage.end())
return nullptr;
return it->second.get();
return it->second;
}
std::vector<FutureSet> PreparedSets::getByTreeHash(IAST::Hash ast_hash) const
PreparedSets::Subqueries PreparedSets::getSubqueries()
{
std::vector<FutureSet> res;
for (const auto & it : this->sets)
{
if (it.first.ast_hash == ast_hash)
res.push_back(it.second);
}
PreparedSets::Subqueries res;
res.reserve(sets_from_subqueries.size());
for (auto & [_, set] : sets_from_subqueries)
res.push_back(set);
return res;
}
PreparedSets::SubqueriesForSets PreparedSets::detachSubqueries()
{
auto res = std::move(subqueries);
subqueries = SubqueriesForSets();
return res;
}
bool PreparedSets::empty() const { return sets.empty(); }
void SubqueryForSet::createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_)
{
source = std::make_unique<QueryPlan>();
interpreter.buildQueryPlan(*source);
if (table_)
table = table_;
}
bool SubqueryForSet::hasSource() const
{
return source != nullptr;
}
QueryPlanPtr SubqueryForSet::detachSource()
{
auto res = std::move(source);
source = nullptr;
return res;
}
FutureSet::FutureSet(SetPtr set)
{
std::promise<SetPtr> promise;
promise.set_value(set);
*this = FutureSet(promise.get_future());
}
bool FutureSet::isReady() const
{
return future_set.valid() &&
future_set.wait_for(std::chrono::seconds(0)) == std::future_status::ready;
}
bool FutureSet::isCreated() const
{
return isReady() && get() != nullptr && get()->isCreated();
}
std::variant<std::promise<SetPtr>, SharedSet> PreparedSetsCache::findOrPromiseToBuild(const String & key)
{
std::lock_guard lock(cache_mutex);

View File

@ -2,14 +2,13 @@
#include <Parsers/IAST.h>
#include <DataTypes/IDataType.h>
#include <future>
#include <memory>
#include <unordered_map>
#include <vector>
#include <DataTypes/DataTypeLowCardinality.h>
#include <future>
#include <Storages/IStorage_fwd.h>
#include <QueryPipeline/SizeLimits.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Interpreters/Context_fwd.h>
#include <Interpreters/SetKeys.h>
namespace DB
{
@ -18,121 +17,158 @@ class QueryPlan;
class Set;
using SetPtr = std::shared_ptr<Set>;
class InterpreterSelectWithUnionQuery;
struct SetKeyColumns;
class IQueryTreeNode;
using QueryTreeNodePtr = std::shared_ptr<IQueryTreeNode>;
struct Settings;
/// This is a structure for prepared sets cache.
/// SetPtr can be taken from cache, so we should pass holder for it.
struct SetAndKey
{
String key;
SetPtr set;
};
using SetAndKeyPtr = std::shared_ptr<SetAndKey>;
/// Represents a set in a query that might be referenced at analysis time and built later during execution.
/// Also it can represent a constant set that is ready to use.
/// At analysis stage the FutureSets are created but not necessarily filled. Then for non-constant sets there
/// must be an explicit step to build them before they can be used.
/// FutureSet objects can be stored in PreparedSets and are not intended to be used from multiple threads.
class FutureSet final
/// Set may be useful for indexes, in this case special ordered set with stored elements is build inplace.
class FutureSet
{
public:
FutureSet() = default;
virtual ~FutureSet() = default;
/// Create FutureSet from an object that will be created in the future.
explicit FutureSet(const std::shared_future<SetPtr> & future_set_) : future_set(future_set_) {}
/// Returns set if set is ready (created and filled) or nullptr if not.
virtual SetPtr get() const = 0;
/// Returns set->getElementsTypes(), even if set is not created yet.
virtual const DataTypes & getTypes() const = 0;
/// If possible, return set with stored elements useful for PK analysis.
virtual SetPtr buildOrderedSetInplace(const ContextPtr & context) = 0;
};
/// Create FutureSet from a ready set.
explicit FutureSet(SetPtr readySet);
using FutureSetPtr = std::shared_ptr<FutureSet>;
/// The set object will be ready in the future, as opposed to 'null' object when FutureSet is default constructed.
bool isValid() const { return future_set.valid(); }
/// Future set from already filled set.
/// Usually it is from StorageSet.
class FutureSetFromStorage final : public FutureSet
{
public:
FutureSetFromStorage(SetPtr set_);
/// The the value of SetPtr is ready, but the set object might not have been filled yet.
bool isReady() const;
/// The set object is ready and filled.
bool isCreated() const;
SetPtr get() const { chassert(isReady()); return future_set.get(); }
SetPtr get() const override;
const DataTypes & getTypes() const override;
SetPtr buildOrderedSetInplace(const ContextPtr &) override;
private:
std::shared_future<SetPtr> future_set;
SetPtr set;
};
/// Information on how to build set for the [GLOBAL] IN section.
class SubqueryForSet
/// Set from tuple is filled as well as set from storage.
/// Additionally, it can be converted to set useful for PK.
class FutureSetFromTuple final : public FutureSet
{
public:
FutureSetFromTuple(Block block, const Settings & settings);
void createSource(InterpreterSelectWithUnionQuery & interpreter, StoragePtr table_ = nullptr);
SetPtr get() const override { return set; }
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
bool hasSource() const;
const DataTypes & getTypes() const override;
/// Returns query plan for the set's source
/// and removes it from SubqueryForSet because we need to build it only once.
std::unique_ptr<QueryPlan> detachSource();
/// Build this set from the result of the subquery.
String key;
SetPtr set_in_progress;
/// After set_in_progress is finished it will be put into promise_to_fill_set and thus all FutureSet's
/// that are referencing this set will be filled.
std::promise<SetPtr> promise_to_fill_set;
FutureSet set = FutureSet{promise_to_fill_set.get_future()};
/// If set, put the result into the table.
/// This is a temporary table for transferring to remote servers for distributed query processing.
StoragePtr table;
/// The source is obtained using the InterpreterSelectQuery subquery.
std::unique_ptr<QueryPlan> source;
private:
SetPtr set;
SetKeyColumns set_key_columns;
};
struct PreparedSetKey
/// Set from subquery can be built inplace for PK or in CreatingSet step.
/// If use_index_for_in_with_subqueries_max_values is reached, set for PK won't be created,
/// but ordinary set would be created instead.
class FutureSetFromSubquery final : public FutureSet
{
/// Prepared sets for tuple literals are indexed by the hash of the tree contents and by the desired
/// data types of set elements (two different Sets can be required for two tuples with the same contents
/// if left hand sides of the IN operators have different types).
static PreparedSetKey forLiteral(const IAST & ast, DataTypes types_);
public:
FutureSetFromSubquery(
String key,
std::unique_ptr<QueryPlan> source_,
StoragePtr external_table_,
FutureSetPtr external_table_set_,
const Settings & settings);
/// Prepared sets for subqueries are indexed only by the AST contents because the type of the resulting
/// set is fully determined by the subquery.
static PreparedSetKey forSubquery(const IAST & ast);
FutureSetFromSubquery(
String key,
QueryTreeNodePtr query_tree_,
const Settings & settings);
IAST::Hash ast_hash;
DataTypes types; /// Empty for subqueries.
SetPtr get() const override;
const DataTypes & getTypes() const override;
SetPtr buildOrderedSetInplace(const ContextPtr & context) override;
bool operator==(const PreparedSetKey & other) const;
std::unique_ptr<QueryPlan> build(const ContextPtr & context);
String toString() const;
QueryTreeNodePtr detachQueryTree() { return std::move(query_tree); }
void setQueryPlan(std::unique_ptr<QueryPlan> source_);
struct Hash
{
UInt64 operator()(const PreparedSetKey & key) const { return key.ast_hash.first; }
};
private:
SetAndKeyPtr set_and_key;
StoragePtr external_table;
FutureSetPtr external_table_set;
std::unique_ptr<QueryPlan> source;
QueryTreeNodePtr query_tree;
};
/// Container for all the sets used in query.
class PreparedSets
{
public:
using SubqueriesForSets = std::unordered_map<String, SubqueryForSet>;
SubqueryForSet & createOrGetSubquery(const String & subquery_id, const PreparedSetKey & key,
SizeLimits set_size_limit, bool transform_null_in);
SubqueryForSet & getSubquery(const String & subquery_id);
using Hash = std::pair<UInt64, UInt64>;
struct Hashing
{
UInt64 operator()(const Hash & key) const { return key.first ^ key.second; }
};
void set(const PreparedSetKey & key, SetPtr set_);
FutureSet getFuture(const PreparedSetKey & key) const;
SetPtr get(const PreparedSetKey & key) const;
using SetsFromTuple = std::unordered_map<Hash, std::vector<std::shared_ptr<FutureSetFromTuple>>, Hashing>;
using SetsFromStorage = std::unordered_map<Hash, std::shared_ptr<FutureSetFromStorage>, Hashing>;
using SetsFromSubqueries = std::unordered_map<Hash, std::shared_ptr<FutureSetFromSubquery>, Hashing>;
/// Get subqueries and clear them.
/// We need to build a plan for subqueries just once. That's why we can clear them after accessing them.
/// SetPtr would still be available for consumers of PreparedSets.
SubqueriesForSets detachSubqueries();
FutureSetPtr addFromStorage(const Hash & key, SetPtr set_);
FutureSetPtr addFromTuple(const Hash & key, Block block, const Settings & settings);
/// Returns all sets that match the given ast hash not checking types
/// Used in KeyCondition and MergeTreeIndexConditionBloomFilter to make non exact match for types in PreparedSetKey
std::vector<FutureSet> getByTreeHash(IAST::Hash ast_hash) const;
FutureSetPtr addFromSubquery(
const Hash & key,
std::unique_ptr<QueryPlan> source,
StoragePtr external_table,
FutureSetPtr external_table_set,
const Settings & settings);
bool empty() const;
FutureSetPtr addFromSubquery(
const Hash & key,
QueryTreeNodePtr query_tree,
const Settings & settings);
FutureSetPtr findTuple(const Hash & key, const DataTypes & types) const;
std::shared_ptr<FutureSetFromStorage> findStorage(const Hash & key) const;
std::shared_ptr<FutureSetFromSubquery> findSubquery(const Hash & key) const;
using Subqueries = std::vector<std::shared_ptr<FutureSetFromSubquery>>;
Subqueries getSubqueries();
const SetsFromTuple & getSetsFromTuple() const { return sets_from_tuple; }
// const SetsFromStorage & getSetsFromStorage() const { return sets_from_storage; }
// const SetsFromSubqueries & getSetsFromSubquery() const { return sets_from_subqueries; }
static String toString(const Hash & key, const DataTypes & types);
private:
std::unordered_map<PreparedSetKey, FutureSet, PreparedSetKey::Hash> sets;
/// This is the information required for building sets
SubqueriesForSets subqueries;
SetsFromTuple sets_from_tuple;
SetsFromStorage sets_from_storage;
SetsFromSubqueries sets_from_subqueries;
};
using PreparedSetsPtr = std::shared_ptr<PreparedSets>;

View File

@ -246,6 +246,7 @@ ProcessList::insert(const String & query_, const IAST * ast, ContextMutablePtr q
priorities.insert(static_cast<int>(settings.priority)),
std::move(thread_group),
query_kind,
settings,
watch_start_nanoseconds));
increaseQueryKindAmount(query_kind);
@ -342,6 +343,7 @@ QueryStatus::QueryStatus(
QueryPriorities::Handle && priority_handle_,
ThreadGroupPtr && thread_group_,
IAST::QueryKind query_kind_,
const Settings & query_settings_,
UInt64 watch_start_nanoseconds)
: WithContext(context_)
, query(query_)
@ -353,9 +355,11 @@ QueryStatus::QueryStatus(
, query_kind(query_kind_)
, num_queries_increment(CurrentMetrics::Query)
{
auto settings = getContext()->getSettings();
limits.max_execution_time = settings.max_execution_time;
overflow_mode = settings.timeout_overflow_mode;
/// We have to pass `query_settings_` to this constructor because we can't use `context_->getSettings().max_execution_time` here:
/// a QueryStatus is created with `ProcessList::mutex` locked (see ProcessList::insert) and calling `context_->getSettings()`
/// would lock the context's lock too, whereas holding two those locks simultaneously is not good.
limits.max_execution_time = query_settings_.max_execution_time;
overflow_mode = query_settings_.timeout_overflow_mode;
}
QueryStatus::~QueryStatus()
@ -589,10 +593,13 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
res.profile_counters = std::make_shared<ProfileEvents::Counters::Snapshot>(thread_group->performance_counters.getPartiallyAtomicSnapshot());
}
if (get_settings && getContext())
if (get_settings)
{
res.query_settings = std::make_shared<Settings>(getContext()->getSettings());
res.current_database = getContext()->getCurrentDatabase();
if (auto ctx = context.lock())
{
res.query_settings = std::make_shared<Settings>(ctx->getSettings());
res.current_database = ctx->getCurrentDatabase();
}
}
return res;
@ -601,12 +608,18 @@ QueryStatusInfo QueryStatus::getInfo(bool get_thread_list, bool get_profile_even
ProcessList::Info ProcessList::getInfo(bool get_thread_list, bool get_profile_events, bool get_settings) const
{
/// We have to copy `processes` first because `process->getInfo()` below can access the context to get the query settings,
/// and it's better not to keep the process list's lock while doing that.
std::vector<QueryStatusPtr> processes_copy;
{
auto lock = safeLock();
processes_copy.assign(processes.begin(), processes.end());
}
Info per_query_infos;
auto lock = safeLock();
per_query_infos.reserve(processes.size());
for (const auto & process : processes)
per_query_infos.reserve(processes_copy.size());
for (const auto & process : processes_copy)
per_query_infos.emplace_back(process->getInfo(get_thread_list, get_profile_events, get_settings));
return per_query_infos;

View File

@ -164,6 +164,7 @@ public:
QueryPriorities::Handle && priority_handle_,
ThreadGroupPtr && thread_group_,
IAST::QueryKind query_kind_,
const Settings & query_settings_,
UInt64 watch_start_nanoseconds);
~QueryStatus();

View File

@ -103,6 +103,21 @@ void NO_INLINE Set::insertFromBlockImplCase(
}
DataTypes Set::getElementTypes(DataTypes types, bool transform_null_in)
{
for (auto & type : types)
{
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(type.get()))
type = low_cardinality_type->getDictionaryType();
if (!transform_null_in)
type = removeNullable(type);
}
return types;
}
void Set::setHeader(const ColumnsWithTypeAndName & header)
{
std::lock_guard lock(rwlock);
@ -152,46 +167,67 @@ void Set::setHeader(const ColumnsWithTypeAndName & header)
extractNestedColumnsAndNullMap(key_columns, null_map);
}
if (fill_set_elements)
{
/// Create empty columns with set values in advance.
/// It is needed because set may be empty, so method 'insertFromBlock' will be never called.
set_elements.reserve(keys_size);
for (const auto & type : set_elements_types)
set_elements.emplace_back(type->createColumn());
}
/// Choose data structure to use for the set.
data.init(data.chooseMethod(key_columns, key_sizes));
}
void Set::fillSetElements()
{
fill_set_elements = true;
set_elements.reserve(keys_size);
for (const auto & type : set_elements_types)
set_elements.emplace_back(type->createColumn());
}
bool Set::insertFromBlock(const ColumnsWithTypeAndName & columns)
{
Columns cols;
cols.reserve(columns.size());
for (const auto & column : columns)
cols.emplace_back(column.column);
return insertFromBlock(cols);
return insertFromColumns(cols);
}
bool Set::insertFromBlock(const Columns & columns)
bool Set::insertFromColumns(const Columns & columns)
{
size_t rows = columns.at(0)->size();
SetKeyColumns holder;
/// Filter to extract distinct values from the block.
if (fill_set_elements)
holder.filter = ColumnUInt8::create(rows);
bool inserted = insertFromColumns(columns, holder);
if (inserted && fill_set_elements)
{
if (max_elements_to_fill && max_elements_to_fill < data.getTotalRowCount())
{
/// Drop filled elementes
fill_set_elements = false;
set_elements.clear();
}
else
appendSetElements(holder);
}
return inserted;
}
bool Set::insertFromColumns(const Columns & columns, SetKeyColumns & holder)
{
std::lock_guard lock(rwlock);
if (data.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method Set::setHeader must be called before Set::insertFromBlock");
ColumnRawPtrs key_columns;
key_columns.reserve(keys_size);
/// The constant columns to the right of IN are not supported directly. For this, they first materialize.
Columns materialized_columns;
holder.key_columns.reserve(keys_size);
holder.materialized_columns.reserve(keys_size);
/// Remember the columns we will work with
for (size_t i = 0; i < keys_size; ++i)
{
materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded());
key_columns.emplace_back(materialized_columns.back().get());
holder.materialized_columns.emplace_back(columns.at(i)->convertToFullIfNeeded());
holder.key_columns.emplace_back(holder.materialized_columns.back().get());
}
size_t rows = columns.at(0)->size();
@ -200,12 +236,7 @@ bool Set::insertFromBlock(const Columns & columns)
ConstNullMapPtr null_map{};
ColumnPtr null_map_holder;
if (!transform_null_in)
null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map);
/// Filter to extract distinct values from the block.
ColumnUInt8::MutablePtr filter;
if (fill_set_elements)
filter = ColumnUInt8::create(rows);
null_map_holder = extractNestedColumnsAndNullMap(holder.key_columns, null_map);
switch (data.type)
{
@ -213,29 +244,34 @@ bool Set::insertFromBlock(const Columns & columns)
break;
#define M(NAME) \
case SetVariants::Type::NAME: \
insertFromBlockImpl(*data.NAME, key_columns, rows, data, null_map, filter ? &filter->getData() : nullptr); \
insertFromBlockImpl(*data.NAME, holder.key_columns, rows, data, null_map, holder.filter ? &holder.filter->getData() : nullptr); \
break;
APPLY_FOR_SET_VARIANTS(M)
#undef M
}
if (fill_set_elements)
{
for (size_t i = 0; i < keys_size; ++i)
{
auto filtered_column = key_columns[i]->filter(filter->getData(), rows);
if (set_elements[i]->empty())
set_elements[i] = filtered_column;
else
set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size());
if (transform_null_in && null_map_holder)
set_elements[i]->insert(Null{});
}
}
return limits.check(data.getTotalRowCount(), data.getTotalByteCount(), "IN-set", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED);
}
void Set::appendSetElements(SetKeyColumns & holder)
{
if (holder.key_columns.size() != keys_size || set_elements.size() != keys_size)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Invalid number of key columns for set. Expected {} got {} and {}",
keys_size, holder.key_columns.size(), set_elements.size());
size_t rows = holder.key_columns.at(0)->size();
for (size_t i = 0; i < keys_size; ++i)
{
auto filtered_column = holder.key_columns[i]->filter(holder.filter->getData(), rows);
if (set_elements[i]->empty())
set_elements[i] = filtered_column;
else
set_elements[i]->insertRangeFrom(*filtered_column, 0, filtered_column->size());
if (transform_null_in && holder.null_map_holder)
set_elements[i]->insert(Null{});
}
}
void Set::checkIsCreated() const
{
if (!is_created.load())
@ -429,6 +465,11 @@ void Set::checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) c
MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<KeyTuplePositionMapping> && indexes_mapping_)
: has_all_keys(set_elements.size() == indexes_mapping_.size()), indexes_mapping(std::move(indexes_mapping_))
{
// std::cerr << "MergeTreeSetIndex::MergeTreeSetIndex "
// << set_elements.size() << ' ' << indexes_mapping.size() << std::endl;
// for (const auto & vv : indexes_mapping)
// std::cerr << vv.key_index << ' ' << vv.tuple_index << std::endl;
::sort(indexes_mapping.begin(), indexes_mapping.end(),
[](const KeyTuplePositionMapping & l, const KeyTuplePositionMapping & r)
{
@ -471,6 +512,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const Columns & set_elements, std::vector<K
BoolMask MergeTreeSetIndex::checkInRange(const std::vector<Range> & key_ranges, const DataTypes & data_types, bool single_point) const
{
size_t tuple_size = indexes_mapping.size();
// std::cerr << "MergeTreeSetIndex::checkInRange " << single_point << ' ' << tuple_size << ' ' << has_all_keys << std::endl;
FieldValues left_point;
FieldValues right_point;

View File

@ -4,6 +4,7 @@
#include <QueryPipeline/SizeLimits.h>
#include <DataTypes/IDataType.h>
#include <Interpreters/SetVariants.h>
#include <Interpreters/SetKeys.h>
#include <Parsers/IAST.h>
#include <Storages/MergeTree/BoolMask.h>
@ -30,9 +31,9 @@ public:
/// (that is useful only for checking that some value is in the set and may not store the original values),
/// store all set elements in explicit form.
/// This is needed for subsequent use for index.
Set(const SizeLimits & limits_, bool fill_set_elements_, bool transform_null_in_)
Set(const SizeLimits & limits_, size_t max_elements_to_fill_, bool transform_null_in_)
: log(&Poco::Logger::get("Set")),
limits(limits_), fill_set_elements(fill_set_elements_), transform_null_in(transform_null_in_)
limits(limits_), max_elements_to_fill(max_elements_to_fill_), transform_null_in(transform_null_in_)
{
}
@ -45,9 +46,13 @@ public:
void setHeader(const ColumnsWithTypeAndName & header);
/// Returns false, if some limit was exceeded and no need to insert more data.
bool insertFromBlock(const Columns & columns);
bool insertFromColumns(const Columns & columns);
bool insertFromBlock(const ColumnsWithTypeAndName & columns);
void fillSetElements();
bool insertFromColumns(const Columns & columns, SetKeyColumns & holder);
void appendSetElements(SetKeyColumns & holder);
/// Call after all blocks were inserted. To get the information that set is already created.
void finishInsert() { is_created = true; }
@ -68,13 +73,15 @@ public:
const DataTypes & getDataTypes() const { return data_types; }
const DataTypes & getElementsTypes() const { return set_elements_types; }
bool hasExplicitSetElements() const { return fill_set_elements; }
bool hasExplicitSetElements() const { return fill_set_elements || (!set_elements.empty() && set_elements.front()->size() == data.getTotalRowCount()); }
Columns getSetElements() const { checkIsCreated(); return { set_elements.begin(), set_elements.end() }; }
void checkColumnsNumber(size_t num_key_columns) const;
bool areTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const;
void checkTypesEqual(size_t set_type_idx, const DataTypePtr & other_type) const;
static DataTypes getElementTypes(DataTypes types, bool transform_null_in);
private:
size_t keys_size = 0;
Sizes key_sizes;
@ -110,7 +117,8 @@ private:
SizeLimits limits;
/// Do we need to additionally store all elements of the set in explicit form for subsequent use for index.
bool fill_set_elements;
bool fill_set_elements = false;
size_t max_elements_to_fill;
/// If true, insert NULL values to set.
bool transform_null_in;

View File

@ -0,0 +1,18 @@
#pragma once
#include <Columns/ColumnsNumber.h>
namespace DB
{
/// Prepared key columns for set which can be added to fill set elements.
/// Used only to upgrade set from tuple.
struct SetKeyColumns
{
/// The constant columns to the right of IN are not supported directly. For this, they first materialize.
ColumnRawPtrs key_columns;
Columns materialized_columns;
ColumnPtr null_map_holder;
ColumnUInt8::MutablePtr filter;
};
}

View File

@ -25,7 +25,6 @@
#include <Interpreters/GatherFunctionQuantileVisitor.h>
#include <Interpreters/RewriteSumIfFunctionVisitor.h>
#include <Interpreters/RewriteArrayExistsFunctionVisitor.h>
#include <Interpreters/OptimizeDateOrDateTimeConverterWithPreimageVisitor.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
@ -678,21 +677,6 @@ void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, ContextPtr context)
RemoveInjectiveFunctionsVisitor(data).visit(query);
}
void optimizeDateFilters(ASTSelectQuery * select_query, const std::vector<TableWithColumnNamesAndTypes> & tables_with_columns, ContextPtr context)
{
/// Predicates in HAVING clause has been moved to WHERE clause.
if (select_query->where())
{
OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context};
OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refWhere());
}
if (select_query->prewhere())
{
OptimizeDateOrDateTimeConverterWithPreimageVisitor::Data data{tables_with_columns, context};
OptimizeDateOrDateTimeConverterWithPreimageVisitor(data).visit(select_query->refPrewhere());
}
}
void transformIfStringsIntoEnum(ASTPtr & query)
{
std::unordered_set<String> function_names = {"if", "transform"};
@ -796,9 +780,6 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result,
tables_with_columns, result.storage_snapshot->metadata, result.storage);
}
/// Rewrite date filters to avoid the calls of converters such as toYear, toYYYYMM, etc.
optimizeDateFilters(select_query, tables_with_columns, context);
/// GROUP BY injective function elimination.
optimizeGroupBy(select_query, context);

View File

@ -6,6 +6,7 @@
#include <Common/SensitiveDataMasker.h>
#include <Interpreters/AsynchronousInsertQueue.h>
#include <Interpreters/Cache/QueryCache.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/LimitReadBuffer.h>
@ -14,7 +15,6 @@
#include <QueryPipeline/BlockIO.h>
#include <Processors/Transforms/CountingTransform.h>
#include <Processors/Transforms/getSourceFromASTInsertQuery.h>
#include <Processors/Transforms/StreamInQueryCacheTransform.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTInsertQuery.h>
@ -645,139 +645,133 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
}
bool can_use_query_cache = settings.use_query_cache && !internal && !ast->as<ASTExplainQuery>();
QueryCachePtr query_cache = context->getQueryCache();
const bool can_use_query_cache = query_cache != nullptr && settings.use_query_cache && !internal && (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>());
bool write_into_query_cache = false;
if (!async_insert)
{
/// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots
if (!context->getCurrentTransaction() && settings.implicit_transaction && !ast->as<ASTTransactionControl>())
/// If it is a non-internal SELECT, and passive/read use of the query cache is enabled, and the cache knows the query, then set
/// a pipeline with a source populated by the query cache.
auto get_result_from_query_cache = [&]()
{
try
if (can_use_query_cache && settings.enable_reads_from_query_cache)
{
if (context->isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot create transactions");
execute_implicit_tcl_query(context, ASTTransactionControl::BEGIN);
}
catch (Exception & e)
{
e.addMessage("while starting a transaction with 'implicit_transaction'");
throw;
}
}
interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal));
const auto & query_settings = context->getSettingsRef();
if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction)
{
if (!interpreter->supportsTransactions())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID());
}
if (!interpreter->ignoreQuota() && !quota_checked)
{
quota = context->getQuota();
if (quota)
{
if (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>())
{
quota->used(QuotaType::QUERY_SELECTS, 1);
}
else if (ast->as<ASTInsertQuery>())
{
quota->used(QuotaType::QUERY_INSERTS, 1);
}
quota->used(QuotaType::QUERIES, 1);
quota->checkExceeded(QuotaType::ERRORS);
}
}
if (!interpreter->ignoreLimits())
{
limits.mode = LimitsMode::LIMITS_CURRENT;
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
}
if (auto * insert_interpreter = typeid_cast<InterpreterInsertQuery *>(&*interpreter))
{
/// Save insertion table (not table function). TODO: support remote() table function.
auto table_id = insert_interpreter->getDatabaseTable();
if (!table_id.empty())
context->setInsertionTable(std::move(table_id));
if (insert_data_buffer_holder)
insert_interpreter->addBuffer(std::move(insert_data_buffer_holder));
}
{
std::unique_ptr<OpenTelemetry::SpanHolder> span;
if (OpenTelemetry::CurrentContext().isTraceEnabled())
{
auto * raw_interpreter_ptr = interpreter.get();
String class_name(demangle(typeid(*raw_interpreter_ptr).name()));
span = std::make_unique<OpenTelemetry::SpanHolder>(class_name + "::execute()");
}
res = interpreter->execute();
/// If
/// - it is a SELECT query,
/// - passive (read) use of the query cache is enabled, and
/// - the query cache knows the query result
/// then replace the pipeline by a new pipeline with a single source that is populated from the query cache
auto query_cache = context->getQueryCache();
bool read_result_from_query_cache = false; /// a query must not read from *and* write to the query cache at the same time
if (query_cache != nullptr
&& (can_use_query_cache && settings.enable_reads_from_query_cache)
&& res.pipeline.pulling())
{
QueryCache::Key key(
ast, res.pipeline.getHeader(),
context->getUserName(), /*dummy for is_shared*/ false,
/*dummy value for expires_at*/ std::chrono::system_clock::from_time_t(1),
/*dummy value for is_compressed*/ false);
QueryCache::Key key(ast, context->getUserName());
QueryCache::Reader reader = query_cache->createReader(key);
if (reader.hasCacheEntryForKey())
{
QueryPipeline pipeline;
pipeline.readFromQueryCache(reader.getSource(), reader.getSourceTotals(), reader.getSourceExtremes());
res.pipeline = std::move(pipeline);
read_result_from_query_cache = true;
return true;
}
}
return false;
};
/// If
/// - it is a SELECT query, and
/// - active (write) use of the query cache is enabled
/// then add a processor on top of the pipeline which stores the result in the query cache.
if (!read_result_from_query_cache
&& query_cache != nullptr
&& can_use_query_cache && settings.enable_writes_to_query_cache
&& res.pipeline.pulling()
&& (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions))
if (!get_result_from_query_cache())
{
/// We need to start the (implicit) transaction before getting the interpreter as this will get links to the latest snapshots
if (!context->getCurrentTransaction() && settings.implicit_transaction && !ast->as<ASTTransactionControl>())
{
QueryCache::Key key(
ast, res.pipeline.getHeader(),
context->getUserName(), settings.query_cache_share_between_users,
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl),
settings.query_cache_compress_entries);
const size_t num_query_runs = query_cache->recordQueryRun(key);
if (num_query_runs > settings.query_cache_min_query_runs)
try
{
auto query_cache_writer = std::make_shared<QueryCache::Writer>(query_cache->createWriter(
key,
std::chrono::milliseconds(settings.query_cache_min_query_duration.totalMilliseconds()),
settings.query_cache_squash_partial_results,
settings.max_block_size,
settings.query_cache_max_size_in_bytes,
settings.query_cache_max_entries));
res.pipeline.writeResultIntoQueryCache(query_cache_writer);
if (context->isGlobalContext())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Global context cannot create transactions");
execute_implicit_tcl_query(context, ASTTransactionControl::BEGIN);
}
catch (Exception & e)
{
e.addMessage("while starting a transaction with 'implicit_transaction'");
throw;
}
}
interpreter = InterpreterFactory::get(ast, context, SelectQueryOptions(stage).setInternal(internal));
const auto & query_settings = context->getSettingsRef();
if (context->getCurrentTransaction() && query_settings.throw_on_unsupported_query_inside_transaction)
{
if (!interpreter->supportsTransactions())
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transactions are not supported for this type of query ({})", ast->getID());
}
if (!interpreter->ignoreQuota() && !quota_checked)
{
quota = context->getQuota();
if (quota)
{
if (ast->as<ASTSelectQuery>() || ast->as<ASTSelectWithUnionQuery>())
{
quota->used(QuotaType::QUERY_SELECTS, 1);
}
else if (ast->as<ASTInsertQuery>())
{
quota->used(QuotaType::QUERY_INSERTS, 1);
}
quota->used(QuotaType::QUERIES, 1);
quota->checkExceeded(QuotaType::ERRORS);
}
}
if (!interpreter->ignoreLimits())
{
limits.mode = LimitsMode::LIMITS_CURRENT;
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
}
if (auto * insert_interpreter = typeid_cast<InterpreterInsertQuery *>(&*interpreter))
{
/// Save insertion table (not table function). TODO: support remote() table function.
auto table_id = insert_interpreter->getDatabaseTable();
if (!table_id.empty())
context->setInsertionTable(std::move(table_id));
if (insert_data_buffer_holder)
insert_interpreter->addBuffer(std::move(insert_data_buffer_holder));
}
{
std::unique_ptr<OpenTelemetry::SpanHolder> span;
if (OpenTelemetry::CurrentContext().isTraceEnabled())
{
auto * raw_interpreter_ptr = interpreter.get();
String class_name(demangle(typeid(*raw_interpreter_ptr).name()));
span = std::make_unique<OpenTelemetry::SpanHolder>(class_name + "::execute()");
}
res = interpreter->execute();
/// If it is a non-internal SELECT query, and active/write use of the query cache is enabled, then add a processor on
/// top of the pipeline which stores the result in the query cache.
if (can_use_query_cache && settings.enable_writes_to_query_cache
&& (!astContainsNonDeterministicFunctions(ast, context) || settings.query_cache_store_results_of_queries_with_nondeterministic_functions))
{
QueryCache::Key key(
ast, res.pipeline.getHeader(),
context->getUserName(), settings.query_cache_share_between_users,
std::chrono::system_clock::now() + std::chrono::seconds(settings.query_cache_ttl),
settings.query_cache_compress_entries);
const size_t num_query_runs = query_cache->recordQueryRun(key);
if (num_query_runs > settings.query_cache_min_query_runs)
{
auto query_cache_writer = std::make_shared<QueryCache::Writer>(query_cache->createWriter(
key,
std::chrono::milliseconds(settings.query_cache_min_query_duration.totalMilliseconds()),
settings.query_cache_squash_partial_results,
settings.max_block_size,
settings.query_cache_max_size_in_bytes,
settings.query_cache_max_entries));
res.pipeline.writeResultIntoQueryCache(query_cache_writer);
write_into_query_cache = true;
}
}
}
}
}
@ -930,9 +924,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
auto finish_callback = [elem,
context,
ast,
my_can_use_query_cache = can_use_query_cache,
enable_writes_to_query_cache = settings.enable_writes_to_query_cache,
query_cache_store_results_of_queries_with_nondeterministic_functions = settings.query_cache_store_results_of_queries_with_nondeterministic_functions,
write_into_query_cache,
log_queries,
log_queries_min_type = settings.log_queries_min_type,
log_queries_min_query_duration_ms = settings.log_queries_min_query_duration_ms.totalMilliseconds(),
@ -943,16 +935,10 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
pulling_pipeline = pipeline.pulling(),
query_span](QueryPipeline & query_pipeline) mutable
{
/// If active (write) use of the query cache is enabled and the query is eligible for result caching, then store the query
/// result buffered in the special-purpose cache processor (added on top of the pipeline) into the cache.
auto query_cache = context->getQueryCache();
if (query_cache != nullptr
&& pulling_pipeline
&& my_can_use_query_cache && enable_writes_to_query_cache
&& (!astContainsNonDeterministicFunctions(ast, context) || query_cache_store_results_of_queries_with_nondeterministic_functions))
{
if (write_into_query_cache)
/// Trigger the actual write of the buffered query result into the query cache. This is done explicitly to prevent
/// partial/garbage results in case of exceptions during query execution.
query_pipeline.finalizeWriteInQueryCache();
}
QueryStatusPtr process_list_elem = context->getProcessListElement();

View File

@ -8,9 +8,13 @@
#include <Analyzer/Utils.h>
#include <Analyzer/SetUtils.h>
#include <Analyzer/InDepthQueryTreeVisitor.h>
#include <Analyzer/ColumnNode.h>
#include <Analyzer/ConstantNode.h>
#include <Analyzer/FunctionNode.h>
#include <Analyzer/TableNode.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Planner/Planner.h>
namespace DB
{
@ -41,11 +45,7 @@ public:
auto in_second_argument_node_type = in_second_argument->getNodeType();
const auto & settings = planner_context.getQueryContext()->getSettingsRef();
String set_key = planner_context.createSetKey(in_second_argument);
if (planner_context.hasSet(set_key))
return;
auto & sets = planner_context.getPreparedSets();
/// Tables and table functions are replaced with subquery at Analysis stage, except special Set table.
auto * second_argument_table = in_second_argument->as<TableNode>();
@ -54,23 +54,69 @@ public:
if (storage_set)
{
/// Handle storage_set as ready set.
planner_context.registerSet(set_key, PlannerSet(FutureSet(storage_set->getSet())));
auto set_key = in_second_argument->getTreeHash();
sets.addFromStorage(set_key, storage_set->getSet());
}
else if (const auto * constant_node = in_second_argument->as<ConstantNode>())
{
auto set = makeSetForConstantValue(
auto set = getSetElementsForConstantValue(
in_first_argument->getResultType(),
constant_node->getValue(),
constant_node->getResultType(),
settings);
settings.transform_null_in);
planner_context.registerSet(set_key, PlannerSet(FutureSet(std::move(set))));
DataTypes set_element_types = {in_first_argument->getResultType()};
const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(set_element_types.front().get());
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
set_element_types = left_tuple_type->getElements();
set_element_types = Set::getElementTypes(std::move(set_element_types), settings.transform_null_in);
auto set_key = in_second_argument->getTreeHash();
if (sets.findTuple(set_key, set_element_types))
return;
sets.addFromTuple(set_key, std::move(set), settings);
}
else if (in_second_argument_node_type == QueryTreeNodeType::QUERY ||
in_second_argument_node_type == QueryTreeNodeType::UNION ||
in_second_argument_node_type == QueryTreeNodeType::TABLE)
{
planner_context.registerSet(set_key, PlannerSet(in_second_argument));
auto set_key = in_second_argument->getTreeHash();
if (sets.findSubquery(set_key))
return;
auto subquery_to_execute = in_second_argument;
if (auto * table_node = in_second_argument->as<TableNode>())
{
auto storage_snapshot = table_node->getStorageSnapshot();
auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary));
size_t columns_to_select_size = columns_to_select.size();
auto column_nodes_to_select = std::make_shared<ListNode>();
column_nodes_to_select->getNodes().reserve(columns_to_select_size);
NamesAndTypes projection_columns;
projection_columns.reserve(columns_to_select_size);
for (auto & column : columns_to_select)
{
column_nodes_to_select->getNodes().emplace_back(std::make_shared<ColumnNode>(column, subquery_to_execute));
projection_columns.emplace_back(column.name, column.type);
}
auto subquery_for_table = std::make_shared<QueryNode>(Context::createCopy(planner_context.getQueryContext()));
subquery_for_table->setIsSubquery(true);
subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select);
subquery_for_table->getJoinTree() = std::move(subquery_to_execute);
subquery_for_table->resolveProjectionColumns(std::move(projection_columns));
subquery_to_execute = std::move(subquery_for_table);
}
sets.addFromSubquery(set_key, std::move(subquery_to_execute), settings);
}
else
{

View File

@ -7,6 +7,8 @@
namespace DB
{
struct SelectQueryOptions;
/** Collect prepared sets and sets for subqueries that are necessary to execute IN function and its variations.
* Collected sets are registered in planner context.
*/

View File

@ -3,11 +3,13 @@
#include <Core/ProtocolDefines.h>
#include <Common/logger_useful.h>
#include <Common/ProfileEvents.h>
#include <Columns/ColumnSet.h>
#include <DataTypes/DataTypeString.h>
#include <Functions/FunctionFactory.h>
#include <Functions/CastOverloadResolver.h>
#include <Functions/indexHint.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/Sources/SourceFromSingleChunk.h>
@ -894,79 +896,72 @@ void addOffsetStep(QueryPlan & query_plan, const QueryAnalysisResult & query_ana
query_plan.addStep(std::move(offsets_step));
}
void addBuildSubqueriesForSetsStepIfNeeded(QueryPlan & query_plan,
void collectSetsFromActionsDAG(const ActionsDAGPtr & dag, std::unordered_set<const FutureSet *> & useful_sets)
{
for (const auto & node : dag->getNodes())
{
if (node.column)
{
const IColumn * column = node.column.get();
if (const auto * column_const = typeid_cast<const ColumnConst *>(column))
column = &column_const->getDataColumn();
if (const auto * column_set = typeid_cast<const ColumnSet *>(column))
useful_sets.insert(column_set->getData().get());
}
if (node.type == ActionsDAG::ActionType::FUNCTION && node.function_base->getName() == "indexHint")
{
ActionsDAG::NodeRawConstPtrs children;
if (const auto * adaptor = typeid_cast<const FunctionToFunctionBaseAdaptor *>(node.function_base.get()))
{
if (const auto * index_hint = typeid_cast<const FunctionIndexHint *>(adaptor->getFunction().get()))
{
collectSetsFromActionsDAG(index_hint->getActions(), useful_sets);
}
}
}
}
}
void addBuildSubqueriesForSetsStepIfNeeded(
QueryPlan & query_plan,
const SelectQueryOptions & select_query_options,
const PlannerContextPtr & planner_context,
const std::vector<ActionsDAGPtr> & result_actions_to_execute)
{
PreparedSets::SubqueriesForSets subqueries_for_sets;
auto subqueries = planner_context->getPreparedSets().getSubqueries();
std::unordered_set<const FutureSet *> useful_sets;
for (const auto & actions_to_execute : result_actions_to_execute)
collectSetsFromActionsDAG(actions_to_execute, useful_sets);
auto predicate = [&useful_sets](const auto & set) { return !useful_sets.contains(set.get()); };
auto it = std::remove_if(subqueries.begin(), subqueries.end(), std::move(predicate));
subqueries.erase(it, subqueries.end());
for (auto & subquery : subqueries)
{
for (const auto & node : actions_to_execute->getNodes())
{
const auto & set_key = node.result_name;
auto * planner_set = planner_context->getSetOrNull(set_key);
if (!planner_set)
continue;
auto query_tree = subquery->detachQueryTree();
auto subquery_options = select_query_options.subquery();
Planner subquery_planner(
query_tree,
subquery_options,
planner_context->getGlobalPlannerContext());
subquery_planner.buildQueryPlanIfNeeded();
auto subquery_to_execute = planner_set->getSubqueryNode();
if (planner_set->getSet().isCreated() || !subquery_to_execute)
continue;
if (auto * table_node = subquery_to_execute->as<TableNode>())
{
auto storage_snapshot = table_node->getStorageSnapshot();
auto columns_to_select = storage_snapshot->getColumns(GetColumnsOptions(GetColumnsOptions::Ordinary));
size_t columns_to_select_size = columns_to_select.size();
auto column_nodes_to_select = std::make_shared<ListNode>();
column_nodes_to_select->getNodes().reserve(columns_to_select_size);
NamesAndTypes projection_columns;
projection_columns.reserve(columns_to_select_size);
for (auto & column : columns_to_select)
{
column_nodes_to_select->getNodes().emplace_back(std::make_shared<ColumnNode>(column, subquery_to_execute));
projection_columns.emplace_back(column.name, column.type);
}
auto subquery_for_table = std::make_shared<QueryNode>(Context::createCopy(planner_context->getQueryContext()));
subquery_for_table->setIsSubquery(true);
subquery_for_table->getProjectionNode() = std::move(column_nodes_to_select);
subquery_for_table->getJoinTree() = std::move(subquery_to_execute);
subquery_for_table->resolveProjectionColumns(std::move(projection_columns));
subquery_to_execute = std::move(subquery_for_table);
}
auto subquery_options = select_query_options.subquery();
Planner subquery_planner(
subquery_to_execute,
subquery_options,
planner_context->getGlobalPlannerContext());
subquery_planner.buildQueryPlanIfNeeded();
const auto & settings = planner_context->getQueryContext()->getSettingsRef();
SizeLimits size_limits_for_set = {settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode};
bool tranform_null_in = settings.transform_null_in;
auto set = std::make_shared<Set>(size_limits_for_set, false /*fill_set_elements*/, tranform_null_in);
SubqueryForSet subquery_for_set;
subquery_for_set.key = set_key;
subquery_for_set.set_in_progress = set;
subquery_for_set.set = planner_set->getSet();
subquery_for_set.promise_to_fill_set = planner_set->extractPromiseToBuildSet();
subquery_for_set.source = std::make_unique<QueryPlan>(std::move(subquery_planner).extractQueryPlan());
subqueries_for_sets.emplace(set_key, std::move(subquery_for_set));
}
subquery->setQueryPlan(std::make_unique<QueryPlan>(std::move(subquery_planner).extractQueryPlan()));
}
addCreatingSetsStep(query_plan, std::move(subqueries_for_sets), planner_context->getQueryContext());
if (!subqueries.empty())
{
auto step = std::make_unique<DelayedCreatingSetsStep>(
query_plan.getCurrentDataStream(),
std::move(subqueries),
planner_context->getQueryContext());
query_plan.addStep(std::move(step));
}
}
/// Support for `additional_result_filter` setting

View File

@ -16,6 +16,8 @@
#include <DataTypes/DataTypeSet.h>
#include <Common/FieldVisitorToString.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnConst.h>
@ -623,33 +625,67 @@ PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::vi
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::makeSetForInFunction(const QueryTreeNodePtr & node)
{
const auto & function_node = node->as<FunctionNode &>();
auto in_first_argument = function_node.getArguments().getNodes().at(0);
auto in_second_argument = function_node.getArguments().getNodes().at(1);
auto set_key = planner_context->createSetKey(in_second_argument);
const auto & planner_set = planner_context->getSetOrThrow(set_key);
//auto set_key = planner_context->createSetKey(in_second_argument);
DataTypes set_element_types;
auto in_second_argument_node_type = in_second_argument->getNodeType();
bool subquery_or_table =
in_second_argument_node_type == QueryTreeNodeType::QUERY ||
in_second_argument_node_type == QueryTreeNodeType::UNION ||
in_second_argument_node_type == QueryTreeNodeType::TABLE;
FutureSetPtr set;
auto set_key = in_second_argument->getTreeHash();
if (!subquery_or_table)
{
set_element_types = {in_first_argument->getResultType()};
const auto * left_tuple_type = typeid_cast<const DataTypeTuple *>(set_element_types.front().get());
if (left_tuple_type && left_tuple_type->getElements().size() != 1)
set_element_types = left_tuple_type->getElements();
set_element_types = Set::getElementTypes(std::move(set_element_types), planner_context->getQueryContext()->getSettingsRef().transform_null_in);
set = planner_context->getPreparedSets().findTuple(set_key, set_element_types);
}
else
{
set = planner_context->getPreparedSets().findSubquery(set_key);
if (!set)
set = planner_context->getPreparedSets().findStorage(set_key);
}
if (!set)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"No set is registered for key {}",
PreparedSets::toString(set_key, set_element_types));
ColumnWithTypeAndName column;
column.name = set_key;
column.name = planner_context->createSetKey(in_second_argument);
column.type = std::make_shared<DataTypeSet>();
bool set_is_created = planner_set.getSet().isCreated();
auto column_set = ColumnSet::create(1, planner_set.getSet());
bool set_is_created = set->get() != nullptr;
auto column_set = ColumnSet::create(1, std::move(set));
if (set_is_created)
column.column = ColumnConst::create(std::move(column_set), 1);
else
column.column = std::move(column_set);
actions_stack[0].addConstantIfNecessary(set_key, column);
actions_stack[0].addConstantIfNecessary(column.name, column);
size_t actions_stack_size = actions_stack.size();
for (size_t i = 1; i < actions_stack_size; ++i)
{
auto & actions_stack_node = actions_stack[i];
actions_stack_node.addInputConstantColumnIfNecessary(set_key, column);
actions_stack_node.addInputConstantColumnIfNecessary(column.name, column);
}
return {set_key, 0};
return {column.name, 0};
}
PlannerActionsVisitorImpl::NodeNameAndNodeMinLevel PlannerActionsVisitorImpl::visitIndexHintFunction(const QueryTreeNodePtr & node)

View File

@ -118,50 +118,4 @@ PlannerContext::SetKey PlannerContext::createSetKey(const QueryTreeNodePtr & set
return "__set_" + toString(set_source_hash.first) + '_' + toString(set_source_hash.second);
}
void PlannerContext::registerSet(const SetKey & key, PlannerSet planner_set)
{
if (!planner_set.getSet().isValid())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Set must be initialized");
const auto & subquery_node = planner_set.getSubqueryNode();
if (subquery_node)
{
auto node_type = subquery_node->getNodeType();
if (node_type != QueryTreeNodeType::QUERY &&
node_type != QueryTreeNodeType::UNION &&
node_type != QueryTreeNodeType::TABLE)
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid node for set table expression. Expected query or union. Actual {}",
subquery_node->formatASTForErrorMessage());
}
set_key_to_set.emplace(key, std::move(planner_set));
}
bool PlannerContext::hasSet(const SetKey & key) const
{
return set_key_to_set.contains(key);
}
const PlannerSet & PlannerContext::getSetOrThrow(const SetKey & key) const
{
auto it = set_key_to_set.find(key);
if (it == set_key_to_set.end())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"No set is registered for key {}",
key);
return it->second;
}
PlannerSet * PlannerContext::getSetOrNull(const SetKey & key)
{
auto it = set_key_to_set.find(key);
if (it == set_key_to_set.end())
return nullptr;
return &it->second;
}
}

View File

@ -44,52 +44,6 @@ private:
using GlobalPlannerContextPtr = std::shared_ptr<GlobalPlannerContext>;
/** PlannerSet is wrapper around Set that is used during query planning.
*
* If subquery node is null, such set is already prepared for execution.
*
* If subquery node is not null, then set must be build from the result of the subquery.
* If subquery node is not null, it must have QUERY or UNION type.
*/
class PlannerSet
{
public:
/// Construct planner set that is ready for execution
explicit PlannerSet(FutureSet set_)
: set(std::move(set_))
{}
/// Construct planner set with set and subquery node
explicit PlannerSet(QueryTreeNodePtr subquery_node_)
: set(promise_to_build_set.get_future())
, subquery_node(std::move(subquery_node_))
{}
/// Get a reference to a set that might be not built yet
const FutureSet & getSet() const
{
return set;
}
/// Get subquery node
const QueryTreeNodePtr & getSubqueryNode() const
{
return subquery_node;
}
/// This promise will be fulfilled when set is built and all FutureSet objects will become ready
std::promise<SetPtr> extractPromiseToBuildSet()
{
return std::move(promise_to_build_set);
}
private:
std::promise<SetPtr> promise_to_build_set;
FutureSet set;
QueryTreeNodePtr subquery_node;
};
class PlannerContext
{
public:
@ -177,28 +131,10 @@ public:
using SetKey = std::string;
using SetKeyToSet = std::unordered_map<String, PlannerSet>;
/// Create set key for set source node
static SetKey createSetKey(const QueryTreeNodePtr & set_source_node);
/// Register set for set key
void registerSet(const SetKey & key, PlannerSet planner_set);
/// Returns true if set is registered for key, false otherwise
bool hasSet(const SetKey & key) const;
/// Get set for key, if no set is registered logical exception is thrown
const PlannerSet & getSetOrThrow(const SetKey & key) const;
/// Get set for key, if no set is registered null is returned
PlannerSet * getSetOrNull(const SetKey & key);
/// Get registered sets
const SetKeyToSet & getRegisteredSets() const
{
return set_key_to_set;
}
PreparedSets & getPreparedSets() { return prepared_sets; }
private:
/// Query context
@ -214,8 +150,7 @@ private:
std::unordered_map<QueryTreeNodePtr, TableExpressionData> table_expression_node_to_data;
/// Set key to set
SetKeyToSet set_key_to_set;
PreparedSets prepared_sets;
};
using PlannerContextPtr = std::shared_ptr<PlannerContext>;

View File

@ -105,7 +105,7 @@ CreateSetAndFilterOnTheFlyStep::CreateSetAndFilterOnTheFlyStep(
: ITransformingStep(input_stream_, input_stream_.header, getTraits())
, column_names(column_names_)
, max_rows_in_set(max_rows_in_set_)
, own_set(std::make_shared<SetWithState>(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), false, true))
, own_set(std::make_shared<SetWithState>(SizeLimits(max_rows_in_set, 0, OverflowMode::BREAK), 0, true))
, filtering_set(nullptr)
, crosswise_connection(crosswise_connection_)
, position(position_)

View File

@ -1,6 +1,8 @@
#include <exception>
#include <Processors/QueryPlan/CreatingSetsStep.h>
#include <Processors/QueryPlan/QueryPlan.h>
//#include <Processors/QueryPlan/BuildQueryPipelineSettings.h>
#include <Processors/QueryPlan/Optimizations/QueryPlanOptimizationSettings.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
#include <Processors/Transforms/CreatingSetsTransform.h>
#include <IO/Operators.h>
@ -34,21 +36,21 @@ static ITransformingStep::Traits getTraits()
CreatingSetStep::CreatingSetStep(
const DataStream & input_stream_,
String description_,
SubqueryForSet subquery_for_set_,
SetAndKeyPtr set_and_key_,
StoragePtr external_table_,
SizeLimits network_transfer_limits_,
ContextPtr context_)
: ITransformingStep(input_stream_, Block{}, getTraits())
, WithContext(context_)
, description(std::move(description_))
, subquery_for_set(std::move(subquery_for_set_))
, set_and_key(std::move(set_and_key_))
, external_table(std::move(external_table_))
, network_transfer_limits(std::move(network_transfer_limits_))
, context(std::move(context_))
{
}
void CreatingSetStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(subquery_for_set), network_transfer_limits, getContext());
pipeline.addCreatingSetsTransform(getOutputStream().header, std::move(set_and_key), std::move(external_table), network_transfer_limits, context->getPreparedSetsCache());
}
void CreatingSetStep::updateOutputStream()
@ -61,16 +63,16 @@ void CreatingSetStep::describeActions(FormatSettings & settings) const
String prefix(settings.offset, ' ');
settings.out << prefix;
if (subquery_for_set.set_in_progress)
if (set_and_key->set)
settings.out << "Set: ";
settings.out << description << '\n';
settings.out << set_and_key->key << '\n';
}
void CreatingSetStep::describeActions(JSONBuilder::JSONMap & map) const
{
if (subquery_for_set.set_in_progress)
map.add("Set", description);
if (set_and_key->set)
map.add("Set", set_and_key->key);
}
@ -122,7 +124,7 @@ void CreatingSetsStep::describePipeline(FormatSettings & settings) const
IQueryPlanStep::describePipeline(processors, settings);
}
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context)
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subqueries, ContextPtr context)
{
DataStreams input_streams;
input_streams.emplace_back(query_plan.getCurrentDataStream());
@ -131,26 +133,14 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets
plans.emplace_back(std::make_unique<QueryPlan>(std::move(query_plan)));
query_plan = QueryPlan();
for (auto & [description, subquery_for_set] : subqueries_for_sets)
for (auto & future_set : subqueries)
{
if (!subquery_for_set.hasSource())
{
subquery_for_set.promise_to_fill_set.set_exception(std::make_exception_ptr(
Exception(ErrorCodes::LOGICAL_ERROR, "Subquery for set {} has no source", subquery_for_set.key)));
if (future_set->get())
continue;
}
auto plan = subquery_for_set.detachSource();
const Settings & settings = context->getSettingsRef();
auto creating_set = std::make_unique<CreatingSetStep>(
plan->getCurrentDataStream(),
description,
std::move(subquery_for_set),
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode),
context);
creating_set->setStepDescription("Create set for subquery");
plan->addStep(std::move(creating_set));
auto plan = future_set->build(context);
if (!plan)
continue;
input_streams.emplace_back(plan->getCurrentDataStream());
plans.emplace_back(std::move(plan));
@ -167,12 +157,52 @@ void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets
query_plan.unitePlans(std::move(creating_sets), std::move(plans));
}
std::vector<std::unique_ptr<QueryPlan>> DelayedCreatingSetsStep::makePlansForSets(DelayedCreatingSetsStep && step)
{
std::vector<std::unique_ptr<QueryPlan>> plans;
for (auto & future_set : step.subqueries)
{
if (future_set->get())
continue;
auto plan = future_set->build(step.context);
if (!plan)
continue;
plan->optimize(QueryPlanOptimizationSettings::fromContext(step.context));
plans.emplace_back(std::move(plan));
}
return plans;
}
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context)
{
if (!prepared_sets || prepared_sets->empty())
if (!prepared_sets)
return;
addCreatingSetsStep(query_plan, prepared_sets->detachSubqueries(), context);
auto subqueries = prepared_sets->getSubqueries();
if (subqueries.empty())
return;
addCreatingSetsStep(query_plan, std::move(subqueries), context);
}
DelayedCreatingSetsStep::DelayedCreatingSetsStep(
DataStream input_stream, PreparedSets::Subqueries subqueries_, ContextPtr context_)
: subqueries(std::move(subqueries_)), context(std::move(context_))
{
input_streams = {input_stream};
output_stream = std::move(input_stream);
}
QueryPipelineBuilderPtr DelayedCreatingSetsStep::updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &)
{
throw Exception(
ErrorCodes::LOGICAL_ERROR,
"Cannot build pipeline in DelayedCreatingSets. This step should be optimized out.");
}
}

View File

@ -9,15 +9,15 @@ namespace DB
{
/// Creates sets for subqueries and JOIN. See CreatingSetsTransform.
class CreatingSetStep : public ITransformingStep, WithContext
class CreatingSetStep : public ITransformingStep
{
public:
CreatingSetStep(
const DataStream & input_stream_,
String description_,
SubqueryForSet subquery_for_set_,
SizeLimits network_transfer_limits_,
ContextPtr context_);
const DataStream & input_stream_,
SetAndKeyPtr set_and_key_,
StoragePtr external_table_,
SizeLimits network_transfer_limits_,
ContextPtr context_);
String getName() const override { return "CreatingSet"; }
@ -29,9 +29,10 @@ public:
private:
void updateOutputStream() override;
String description;
SubqueryForSet subquery_for_set;
SetAndKeyPtr set_and_key;
StoragePtr external_table;
SizeLimits network_transfer_limits;
ContextPtr context;
};
class CreatingSetsStep : public IQueryPlanStep
@ -46,7 +47,28 @@ public:
void describePipeline(FormatSettings & settings) const override;
};
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::SubqueriesForSets subqueries_for_sets, ContextPtr context);
/// This is a temporary step which is converted to CreatingSetStep after plan optimization.
/// Can't be used by itself.
class DelayedCreatingSetsStep final : public IQueryPlanStep
{
public:
DelayedCreatingSetsStep(DataStream input_stream, PreparedSets::Subqueries subqueries_, ContextPtr context_);
String getName() const override { return "DelayedCreatingSets"; }
QueryPipelineBuilderPtr updatePipeline(QueryPipelineBuilders, const BuildQueryPipelineSettings &) override;
static std::vector<std::unique_ptr<QueryPlan>> makePlansForSets(DelayedCreatingSetsStep && step);
ContextPtr getContext() const { return context; }
PreparedSets::Subqueries detachSets() { return std::move(subqueries); }
private:
PreparedSets::Subqueries subqueries;
ContextPtr context;
};
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSets::Subqueries subqueries, ContextPtr context);
void addCreatingSetsStep(QueryPlan & query_plan, PreparedSetsPtr prepared_sets, ContextPtr context);

View File

@ -10,6 +10,9 @@
namespace DB
{
class PreparedSets;
using PreparedSetsPtr = std::shared_ptr<PreparedSets>;
std::unique_ptr<QueryPlan> createLocalPlan(
const ASTPtr & query_ast,
const Block & header,

View File

@ -110,6 +110,7 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes);
void optimizeAggregationInOrder(QueryPlan::Node & node, QueryPlan::Nodes &);
bool optimizeUseAggregateProjections(QueryPlan::Node & node, QueryPlan::Nodes & nodes);
bool optimizeUseNormalProjections(Stack & stack, QueryPlan::Nodes & nodes);
bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes);
/// Enable memory bound merging of aggregation states for remote queries
/// in case it was enabled for local plan

View File

@ -0,0 +1,35 @@
#include <memory>
#include <Processors/QueryPlan/Optimizations/Optimizations.h>
#include <Processors/QueryPlan/CreatingSetsStep.h>
#include <Common/typeid_cast.h>
namespace DB::QueryPlanOptimizations
{
bool addPlansForSets(QueryPlan::Node & node, QueryPlan::Nodes & nodes)
{
auto * delayed = typeid_cast<DelayedCreatingSetsStep *>(node.step.get());
if (!delayed)
return false;
auto plans = DelayedCreatingSetsStep::makePlansForSets(std::move(*delayed));
node.children.reserve(1 + plans.size());
DataStreams input_streams;
input_streams.reserve(1 + plans.size());
input_streams.push_back(node.children.front()->step->getOutputStream());
for (const auto & plan : plans)
{
input_streams.push_back(plan->getCurrentDataStream());
node.children.push_back(plan->getRootNode());
nodes.splice(nodes.end(), QueryPlan::detachNodes(std::move(*plan)));
}
auto creating_sets = std::make_unique<CreatingSetsStep>(std::move(input_streams));
creating_sets->setStepDescription("Create sets before main query execution");
node.step = std::move(creating_sets);
return true;
}
}

View File

@ -268,6 +268,19 @@ size_t tryPushDownFilter(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes
return 2;
}
if (auto * delayed = typeid_cast<DelayedCreatingSetsStep *>(child.get()))
{
/// CreatingSets does not change header.
/// We can push down filter and update header.
/// Filter - DelayedCreatingSets - Something
child = std::make_unique<DelayedCreatingSetsStep>(filter->getOutputStream(), delayed->detachSets(), delayed->getContext());
std::swap(parent, child);
std::swap(parent_node->children, child_node->children);
std::swap(parent_node->children.front(), child_node->children.front());
/// DelayedCreatingSets - Filter - Something
return 2;
}
if (auto * totals_having = typeid_cast<TotalsHavingStep *>(child.get()))
{
/// If totals step has HAVING expression, skip it for now.

View File

@ -28,6 +28,8 @@ void optimizePrimaryKeyCondition(const Stack & stack)
else
break;
}
source_step_with_filter->onAddFilterFinish();
}
}

View File

@ -167,6 +167,7 @@ void optimizeTreeSecondPass(const QueryPlanOptimizationSettings & optimization_s
optimizePrewhere(stack, nodes);
optimizePrimaryKeyCondition(stack);
enableMemoryBoundMerging(*stack.back().node, nodes);
addPlansForSets(*stack.back().node, nodes);
stack.pop_back();
}

View File

@ -541,4 +541,9 @@ void QueryPlan::explainEstimate(MutableColumns & columns)
}
}
QueryPlan::Nodes QueryPlan::detachNodes(QueryPlan && plan)
{
return std::move(plan.nodes);
}
}

View File

@ -105,10 +105,11 @@ public:
std::vector<Node *> children = {};
};
const Node * getRootNode() const { return root; }
using Nodes = std::list<Node>;
Node * getRootNode() const { return root; }
static Nodes detachNodes(QueryPlan && plan);
private:
QueryPlanResourceHolder resources;
Nodes nodes;

View File

@ -96,11 +96,13 @@ private:
};
ReadFromMemoryStorageStep::ReadFromMemoryStorageStep(const Names & columns_to_read_,
StoragePtr storage_,
const StorageSnapshotPtr & storage_snapshot_,
const size_t num_streams_,
const bool delay_read_for_global_sub_queries_) :
SourceStepWithFilter(DataStream{.header=storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}),
columns_to_read(columns_to_read_),
storage(std::move(storage_)),
storage_snapshot(storage_snapshot_),
num_streams(num_streams_),
delay_read_for_global_sub_queries(delay_read_for_global_sub_queries_)
@ -142,9 +144,9 @@ Pipe ReadFromMemoryStorageStep::makePipe()
storage_snapshot,
nullptr /* data */,
nullptr /* parallel execution index */,
[current_data](std::shared_ptr<const Blocks> & data_to_initialize)
[my_storage = storage](std::shared_ptr<const Blocks> & data_to_initialize)
{
data_to_initialize = current_data;
data_to_initialize = assert_cast<const StorageMemory &>(*my_storage).data.get();
}));
}

View File

@ -16,6 +16,7 @@ class ReadFromMemoryStorageStep final : public SourceStepWithFilter
{
public:
ReadFromMemoryStorageStep(const Names & columns_to_read_,
StoragePtr storage_,
const StorageSnapshotPtr & storage_snapshot_,
size_t num_streams_,
bool delay_read_for_global_sub_queries_);
@ -35,6 +36,7 @@ private:
static constexpr auto name = "ReadFromMemoryStorage";
Names columns_to_read;
StoragePtr storage;
StorageSnapshotPtr storage_snapshot;
size_t num_streams;
bool delay_read_for_global_sub_queries;

View File

@ -37,6 +37,8 @@
#include <Common/JSONBuilder.h>
#include <Common/isLocalAddress.h>
#include <Common/logger_useful.h>
#include <Parsers/parseIdentifierOrStringLiteral.h>
#include <Parsers/ExpressionListParsers.h>
#include <algorithm>
#include <functional>
@ -100,6 +102,7 @@ namespace ErrorCodes
extern const int INDEX_NOT_USED;
extern const int LOGICAL_ERROR;
extern const int TOO_MANY_ROWS;
extern const int CANNOT_PARSE_TEXT;
}
static MergeTreeReaderSettings getMergeTreeReaderSettings(
@ -249,7 +252,7 @@ ReadFromMergeTree::ReadFromMergeTree(
{ /// build sort description for output stream
SortDescription sort_description;
const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns();
const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns();
const Block & header = output_stream->header;
const int sort_direction = getSortDirection();
for (const auto & column_name : sorting_key_columns)
@ -1122,7 +1125,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
prewhere_info,
filter_nodes,
storage_snapshot->metadata,
storage_snapshot->getMetadataForQuery(),
metadata_for_reading,
query_info,
context,
requested_num_streams,
@ -1130,7 +1133,179 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
data,
real_column_names,
sample_factor_column_queried,
log);
log,
indexes);
}
static ActionsDAGPtr buildFilterDAG(
const ContextPtr & context,
const PrewhereInfoPtr & prewhere_info,
const ActionDAGNodes & added_filter_nodes,
const SelectQueryInfo & query_info)
{
const auto & settings = context->getSettingsRef();
ActionsDAG::NodeRawConstPtrs nodes;
if (prewhere_info)
{
{
const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name);
nodes.push_back(&node);
}
if (prewhere_info->row_level_filter)
{
const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name);
nodes.push_back(&node);
}
}
for (const auto & node : added_filter_nodes.nodes)
nodes.push_back(node);
std::unordered_map<std::string, ColumnWithTypeAndName> node_name_to_input_node_column;
if (settings.allow_experimental_analyzer && query_info.planner_context)
{
const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression);
for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName())
{
const auto & column = table_expression_data.getColumnOrThrow(column_name);
node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name));
}
}
return ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context);
}
static void buildIndexes(
std::optional<ReadFromMergeTree::Indexes> & indexes,
ActionsDAGPtr filter_actions_dag,
const MergeTreeData & data,
const ContextPtr & context,
const SelectQueryInfo & query_info,
const StorageMetadataPtr & metadata_snapshot)
{
indexes.reset();
// Build and check if primary key is used when necessary
const auto & primary_key = metadata_snapshot->getPrimaryKey();
const Names & primary_key_column_names = primary_key.column_names;
const auto & settings = context->getSettingsRef();
if (settings.query_plan_optimize_primary_key)
{
NameSet array_join_name_set;
if (query_info.syntax_analyzer_result)
array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet();
indexes.emplace(ReadFromMergeTree::Indexes{{
filter_actions_dag,
context,
primary_key_column_names,
primary_key.expression,
array_join_name_set}, {}, {}, {}, false});
}
else
{
indexes.emplace(ReadFromMergeTree::Indexes{{
query_info,
context,
primary_key_column_names,
primary_key.expression}, {}, {}, {}, false});
}
if (metadata_snapshot->hasPartitionKey())
{
const auto & partition_key = metadata_snapshot->getPartitionKey();
auto minmax_columns_names = data.getMinMaxColumnsNames(partition_key);
auto minmax_expression_actions = data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context));
indexes->minmax_idx_condition.emplace(filter_actions_dag, context, minmax_columns_names, minmax_expression_actions, NameSet());
indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */);
}
indexes->use_skip_indexes = settings.use_skip_indexes;
bool final = query_info.isFinal();
if (final && !settings.use_skip_indexes_if_final)
indexes->use_skip_indexes = false;
if (!indexes->use_skip_indexes)
return;
const SelectQueryInfo * info = &query_info;
std::optional<SelectQueryInfo> info_copy;
if (settings.allow_experimental_analyzer)
{
info_copy.emplace(query_info);
info_copy->filter_actions_dag = filter_actions_dag;
info = &*info_copy;
}
std::unordered_set<std::string> ignored_index_names;
if (settings.ignore_data_skipping_indices.changed)
{
const auto & indices = settings.ignore_data_skipping_indices.toString();
Tokens tokens(indices.data(), indices.data() + indices.size(), settings.max_query_size);
IParser::Pos pos(tokens, static_cast<unsigned>(settings.max_parser_depth));
Expected expected;
/// Use an unordered list rather than string vector
auto parse_single_id_or_literal = [&]
{
String str;
if (!parseIdentifierOrStringLiteral(pos, expected, str))
return false;
ignored_index_names.insert(std::move(str));
return true;
};
if (!ParserList::parseUtil(pos, expected, parse_single_id_or_literal, false))
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices);
}
UsefulSkipIndexes skip_indexes;
using Key = std::pair<String, size_t>;
std::map<Key, size_t> merged;
for (const auto & index : metadata_snapshot->getSecondaryIndices())
{
if (!ignored_index_names.contains(index.name))
{
auto index_helper = MergeTreeIndexFactory::instance().get(index);
if (index_helper->isMergeable())
{
auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size());
if (inserted)
{
skip_indexes.merged_indices.emplace_back();
skip_indexes.merged_indices.back().condition = index_helper->createIndexMergedCondition(*info, metadata_snapshot);
}
skip_indexes.merged_indices[it->second].addIndex(index_helper);
}
else
{
auto condition = index_helper->createIndexCondition(*info, context);
if (!condition->alwaysUnknownOrTrue())
skip_indexes.useful_indices.emplace_back(index_helper, condition);
}
}
}
indexes->skip_indexes = std::move(skip_indexes);
}
void ReadFromMergeTree::onAddFilterFinish()
{
if (!filter_nodes.nodes.empty())
{
auto filter_actions_dag = buildFilterDAG(context, prewhere_info, filter_nodes, query_info);
buildIndexes(indexes, filter_actions_dag, data, context, query_info, metadata_for_reading);
}
}
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
@ -1147,44 +1322,14 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
const MergeTreeData & data,
const Names & real_column_names,
bool sample_factor_column_queried,
Poco::Logger * log)
Poco::Logger * log,
std::optional<Indexes> & indexes)
{
const auto & settings = context->getSettingsRef();
if (settings.allow_experimental_analyzer || settings.query_plan_optimize_primary_key)
{
ActionsDAG::NodeRawConstPtrs nodes;
if (prewhere_info)
{
{
const auto & node = prewhere_info->prewhere_actions->findInOutputs(prewhere_info->prewhere_column_name);
nodes.push_back(&node);
}
if (prewhere_info->row_level_filter)
{
const auto & node = prewhere_info->row_level_filter->findInOutputs(prewhere_info->row_level_column_name);
nodes.push_back(&node);
}
}
for (const auto & node : added_filter_nodes.nodes)
nodes.push_back(node);
std::unordered_map<std::string, ColumnWithTypeAndName> node_name_to_input_node_column;
if (settings.allow_experimental_analyzer && query_info.planner_context)
{
const auto & table_expression_data = query_info.planner_context->getTableExpressionDataOrThrow(query_info.table_expression);
for (const auto & [column_identifier, column_name] : table_expression_data.getColumnIdentifierToColumnName())
{
const auto & column = table_expression_data.getColumnOrThrow(column_name);
node_name_to_input_node_column.emplace(column_identifier, ColumnWithTypeAndName(column.type, column_name));
}
}
auto updated_query_info_with_filter_dag = query_info;
updated_query_info_with_filter_dag.filter_actions_dag = ActionsDAG::buildFilterActionsDAG(nodes, node_name_to_input_node_column, context);
updated_query_info_with_filter_dag.filter_actions_dag = buildFilterDAG(context, prewhere_info, added_filter_nodes, query_info);
return selectRangesToReadImpl(
std::move(parts),
@ -1198,7 +1343,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
data,
real_column_names,
sample_factor_column_queried,
log);
log,
indexes);
}
return selectRangesToReadImpl(
@ -1213,7 +1359,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
data,
real_column_names,
sample_factor_column_queried,
log);
log,
indexes);
}
MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
@ -1228,7 +1375,8 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
const MergeTreeData & data,
const Names & real_column_names,
bool sample_factor_column_queried,
Poco::Logger * log)
Poco::Logger * log,
std::optional<Indexes> & indexes)
{
AnalysisResult result;
const auto & settings = context->getSettingsRef();
@ -1249,31 +1397,14 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
result.column_names_to_read.push_back(ExpressionActions::getSmallestColumn(available_real_columns).name);
}
// storage_snapshot->check(result.column_names_to_read);
// Build and check if primary key is used when necessary
const auto & primary_key = metadata_snapshot->getPrimaryKey();
const Names & primary_key_column_names = primary_key.column_names;
std::optional<KeyCondition> key_condition;
if (settings.query_plan_optimize_primary_key)
{
NameSet array_join_name_set;
if (query_info.syntax_analyzer_result)
array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet();
if (!indexes)
buildIndexes(indexes, query_info.filter_actions_dag, data, context, query_info, metadata_snapshot);
key_condition.emplace(query_info.filter_actions_dag,
context,
primary_key_column_names,
primary_key.expression,
array_join_name_set);
}
else
{
key_condition.emplace(query_info, context, primary_key_column_names, primary_key.expression);
}
if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue())
if (settings.force_primary_key && indexes->key_condition.alwaysUnknownOrTrue())
{
return std::make_shared<MergeTreeDataSelectAnalysisResult>(MergeTreeDataSelectAnalysisResult{
.result = std::make_exception_ptr(Exception(
@ -1281,9 +1412,9 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
"Primary key ({}) is not used and setting 'force_primary_key' is set",
fmt::join(primary_key_column_names, ", ")))});
}
LOG_DEBUG(log, "Key condition: {}", key_condition->toString());
LOG_DEBUG(log, "Key condition: {}", indexes->key_condition.toString());
if (key_condition->alwaysFalse())
if (indexes->key_condition.alwaysFalse())
return std::make_shared<MergeTreeDataSelectAnalysisResult>(MergeTreeDataSelectAnalysisResult{.result = std::move(result)});
size_t total_marks_pk = 0;
@ -1291,12 +1422,13 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
try
{
MergeTreeDataSelectExecutor::filterPartsByPartition(
indexes->partition_pruner,
indexes->minmax_idx_condition,
parts,
alter_conversions,
part_values,
metadata_snapshot_base,
data,
query_info,
context,
max_block_numbers_to_read.get(),
log,
@ -1306,7 +1438,7 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
query_info,
metadata_snapshot->getColumns().getAllPhysical(),
parts,
*key_condition,
indexes->key_condition,
data,
metadata_snapshot,
context,
@ -1322,24 +1454,18 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToReadImpl(
auto reader_settings = getMergeTreeReaderSettings(context, query_info);
bool use_skip_indexes = settings.use_skip_indexes;
bool final = InterpreterSelectQuery::isQueryWithFinal(query_info);
if (final && !settings.use_skip_indexes_if_final)
use_skip_indexes = false;
result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes(
std::move(parts),
std::move(alter_conversions),
metadata_snapshot,
query_info,
context,
*key_condition,
indexes->key_condition,
indexes->skip_indexes,
reader_settings,
log,
num_streams,
result.index_stats,
use_skip_indexes);
indexes->use_skip_indexes);
}
catch (...)
{
@ -1387,7 +1513,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction,
/// Disable read-in-order optimization for reverse order with final.
/// Otherwise, it can lead to incorrect final behavior because the implementation may rely on the reading in direct order).
if (direction != 1 && isQueryWithFinal())
if (direction != 1 && query_info.isFinal())
return false;
auto order_info = std::make_shared<InputOrderInfo>(SortDescription{}, prefix_size, direction, limit);
@ -1405,7 +1531,7 @@ bool ReadFromMergeTree::requestReadingInOrder(size_t prefix_size, int direction,
/// update sort info for output stream
SortDescription sort_description;
const Names & sorting_key_columns = storage_snapshot->getMetadataForQuery()->getSortingKeyColumns();
const Names & sorting_key_columns = metadata_for_reading->getSortingKeyColumns();
const Block & header = output_stream->header;
const int sort_direction = getSortDirection();
for (const auto & column_name : sorting_key_columns)
@ -1515,7 +1641,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::getAnalysisResult() const
bool ReadFromMergeTree::isQueryWithFinal() const
{
return InterpreterSelectQuery::isQueryWithFinal(query_info);
return query_info.isFinal();
}
bool ReadFromMergeTree::isQueryWithSampling() const
@ -1736,6 +1862,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons
processors.emplace_back(processor);
pipeline.init(std::move(pipe));
pipeline.addContext(context);
// Attach QueryIdHolder if needed
if (query_id_holder)
pipeline.setQueryIdHolder(std::move(query_id_holder));

View File

@ -6,6 +6,7 @@
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeReadPool.h>
#include <Storages/MergeTree/AlterConversions.h>
#include <Storages/MergeTree/PartitionPruner.h>
namespace DB
{
@ -25,6 +26,35 @@ struct MergeTreeDataSelectSamplingData
ActionsDAGPtr filter_expression;
};
struct UsefulSkipIndexes
{
struct DataSkippingIndexAndCondition
{
MergeTreeIndexPtr index;
MergeTreeIndexConditionPtr condition;
DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_)
: index(index_), condition(condition_)
{
}
};
struct MergedDataSkippingIndexAndCondition
{
std::vector<MergeTreeIndexPtr> indices;
MergeTreeIndexMergedConditionPtr condition;
void addIndex(const MergeTreeIndexPtr & index)
{
indices.push_back(index);
condition->addIndex(indices.back());
}
};
std::vector<DataSkippingIndexAndCondition> useful_indices;
std::vector<MergedDataSkippingIndexAndCondition> merged_indices;
};
struct MergeTreeDataSelectAnalysisResult;
using MergeTreeDataSelectAnalysisResultPtr = std::shared_ptr<MergeTreeDataSelectAnalysisResult>;
@ -134,6 +164,15 @@ public:
UInt64 getSelectedRows() const { return selected_rows; }
UInt64 getSelectedMarks() const { return selected_marks; }
struct Indexes
{
KeyCondition key_condition;
std::optional<PartitionPruner> partition_pruner;
std::optional<KeyCondition> minmax_idx_condition;
UsefulSkipIndexes skip_indexes;
bool use_skip_indexes;
};
static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(
MergeTreeData::DataPartsVector parts,
std::vector<AlterConversionsPtr> alter_conversions,
@ -148,7 +187,8 @@ public:
const MergeTreeData & data,
const Names & real_column_names,
bool sample_factor_column_queried,
Poco::Logger * log);
Poco::Logger * log,
std::optional<Indexes> & indexes);
MergeTreeDataSelectAnalysisResultPtr selectRangesToRead(
MergeTreeData::DataPartsVector parts,
@ -164,7 +204,6 @@ public:
bool readsInOrder() const;
void updatePrewhereInfo(const PrewhereInfoPtr & prewhere_info_value);
bool isQueryWithFinal() const;
bool isQueryWithSampling() const;
@ -187,6 +226,8 @@ public:
size_t getNumStreams() const { return requested_num_streams; }
bool isParallelReadingEnabled() const { return read_task_callback != std::nullopt; }
void onAddFilterFinish() override;
private:
static MergeTreeDataSelectAnalysisResultPtr selectRangesToReadImpl(
MergeTreeData::DataPartsVector parts,
@ -200,7 +241,8 @@ private:
const MergeTreeData & data,
const Names & real_column_names,
bool sample_factor_column_queried,
Poco::Logger * log);
Poco::Logger * log,
std::optional<Indexes> & indexes);
int getSortDirection() const
{
@ -241,6 +283,9 @@ private:
std::shared_ptr<PartitionIdToMaxBlock> max_block_numbers_to_read;
/// Pre-computed value, needed to trigger sets creating for PK
mutable std::optional<Indexes> indexes;
Poco::Logger * log;
UInt64 selected_parts = 0;
UInt64 selected_rows = 0;

View File

@ -37,6 +37,8 @@ public:
filter_dags.push_back(std::move(filter_dag));
}
virtual void onAddFilterFinish() {}
protected:
std::vector<ActionsDAGPtr> filter_dags;
ActionDAGNodes filter_nodes;

View File

@ -106,7 +106,7 @@ void CreatingSetsOnTheFlyTransform::transform(Chunk & chunk)
if (chunk.getNumRows())
{
Columns key_columns = getColumnsByIndices(chunk, key_column_indices);
bool limit_exceeded = !set->insertFromBlock(key_columns);
bool limit_exceeded = !set->insertFromColumns(key_columns);
if (limit_exceeded)
{
auto prev_state = set->state.exchange(SetWithState::State::Suspended);

View File

@ -25,13 +25,15 @@ CreatingSetsTransform::~CreatingSetsTransform() = default;
CreatingSetsTransform::CreatingSetsTransform(
Block in_header_,
Block out_header_,
SubqueryForSet subquery_for_set_,
SetAndKeyPtr set_and_key_,
StoragePtr external_table_,
SizeLimits network_transfer_limits_,
ContextPtr context_)
PreparedSetsCachePtr prepared_sets_cache_)
: IAccumulatingTransform(std::move(in_header_), std::move(out_header_))
, WithContext(context_)
, subquery(std::move(subquery_for_set_))
, set_and_key(std::move(set_and_key_))
, external_table(std::move(external_table_))
, network_transfer_limits(std::move(network_transfer_limits_))
, prepared_sets_cache(std::move(prepared_sets_cache_))
{
}
@ -52,31 +54,30 @@ void CreatingSetsTransform::work()
void CreatingSetsTransform::startSubquery()
{
/// Lookup the set in the cache if we don't need to build table.
auto ctx = context.lock();
if (ctx && ctx->getPreparedSetsCache() && !subquery.table)
if (prepared_sets_cache && !external_table)
{
/// Try to find the set in the cache and wait for it to be built.
/// Retry if the set from cache fails to be built.
while (true)
{
auto from_cache = ctx->getPreparedSetsCache()->findOrPromiseToBuild(subquery.key);
auto from_cache = prepared_sets_cache->findOrPromiseToBuild(set_and_key->key);
if (from_cache.index() == 0)
{
LOG_TRACE(log, "Building set, key: {}", set_and_key->key);
promise_to_build = std::move(std::get<0>(from_cache));
}
else
{
LOG_TRACE(log, "Waiting for set to be build by another thread, key: {}", subquery.key);
LOG_TRACE(log, "Waiting for set to be build by another thread, key: {}", set_and_key->key);
SharedSet set_built_by_another_thread = std::move(std::get<1>(from_cache));
const SetPtr & ready_set = set_built_by_another_thread.get();
if (!ready_set)
{
LOG_TRACE(log, "Failed to use set from cache, key: {}", subquery.key);
LOG_TRACE(log, "Failed to use set from cache, key: {}", set_and_key->key);
continue;
}
subquery.promise_to_fill_set.set_value(ready_set);
subquery.set_in_progress.reset();
set_and_key->set = ready_set;
done_with_set = true;
set_from_cache = true;
}
@ -84,19 +85,19 @@ void CreatingSetsTransform::startSubquery()
}
}
if (subquery.set_in_progress)
LOG_TRACE(log, "Creating set, key: {}", subquery.key);
if (subquery.table)
if (set_and_key->set && !set_from_cache)
LOG_TRACE(log, "Creating set, key: {}", set_and_key->key);
if (external_table)
LOG_TRACE(log, "Filling temporary table.");
if (subquery.table)
if (external_table)
/// TODO: make via port
table_out = QueryPipeline(subquery.table->write({}, subquery.table->getInMemoryMetadataPtr(), getContext(), /*async_insert=*/false));
table_out = QueryPipeline(external_table->write({}, external_table->getInMemoryMetadataPtr(), nullptr, /*async_insert=*/false));
done_with_set = !subquery.set_in_progress;
done_with_table = !subquery.table;
done_with_set = !set_and_key->set || set_from_cache;
done_with_table = !external_table;
if ((done_with_set && !set_from_cache) /*&& done_with_join*/ && done_with_table)
if ((done_with_set && !set_from_cache) && done_with_table)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: nothing to do with subquery");
if (table_out.initialized())
@ -116,9 +117,9 @@ void CreatingSetsTransform::finishSubquery()
}
else if (read_rows != 0)
{
if (subquery.set_in_progress)
LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", subquery.set_in_progress->getTotalRowCount(), read_rows, seconds);
if (subquery.table)
if (set_and_key->set)
LOG_DEBUG(log, "Created Set with {} entries from {} rows in {} sec.", set_and_key->set->getTotalRowCount(), read_rows, seconds);
if (external_table)
LOG_DEBUG(log, "Created Table with {} rows in {} sec.", read_rows, seconds);
}
else
@ -131,11 +132,6 @@ void CreatingSetsTransform::init()
{
is_initialized = true;
if (subquery.set_in_progress)
{
subquery.set_in_progress->setHeader(getInputPort().getHeader().getColumnsWithTypeAndName());
}
watch.restart();
startSubquery();
}
@ -147,7 +143,7 @@ void CreatingSetsTransform::consume(Chunk chunk)
if (!done_with_set)
{
if (!subquery.set_in_progress->insertFromBlock(block.getColumnsWithTypeAndName()))
if (!set_and_key->set->insertFromBlock(block.getColumnsWithTypeAndName()))
done_with_set = true;
}
@ -170,12 +166,11 @@ void CreatingSetsTransform::consume(Chunk chunk)
Chunk CreatingSetsTransform::generate()
{
if (subquery.set_in_progress)
if (set_and_key->set && !set_from_cache)
{
subquery.set_in_progress->finishInsert();
subquery.promise_to_fill_set.set_value(subquery.set_in_progress);
set_and_key->set->finishInsert();
if (promise_to_build)
promise_to_build->set_value(subquery.set_in_progress);
promise_to_build->set_value(set_and_key->set);
}
if (table_out.initialized())

View File

@ -23,15 +23,16 @@ class PushingPipelineExecutor;
/// Don't return any data. Sets are created when Finish status is returned.
/// In general, several work() methods need to be called to finish.
/// Independent processors is created for each subquery.
class CreatingSetsTransform : public IAccumulatingTransform, WithContext
class CreatingSetsTransform : public IAccumulatingTransform
{
public:
CreatingSetsTransform(
Block in_header_,
Block out_header_,
SubqueryForSet subquery_for_set_,
SetAndKeyPtr set_and_key_,
StoragePtr external_table_,
SizeLimits network_transfer_limits_,
ContextPtr context_);
PreparedSetsCachePtr prepared_sets_cache_);
~CreatingSetsTransform() override;
@ -42,7 +43,8 @@ public:
Chunk generate() override;
private:
SubqueryForSet subquery;
SetAndKeyPtr set_and_key;
StoragePtr external_table;
std::optional<std::promise<SetPtr>> promise_to_build;
QueryPipeline table_out;
@ -55,6 +57,7 @@ private:
bool done_with_table = true;
SizeLimits network_transfer_limits;
PreparedSetsCachePtr prepared_sets_cache;
size_t rows_to_transfer = 0;
size_t bytes_to_transfer = 0;

View File

@ -569,16 +569,22 @@ std::unique_ptr<QueryPipelineBuilder> QueryPipelineBuilder::joinPipelinesRightLe
return left;
}
void QueryPipelineBuilder::addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context)
void QueryPipelineBuilder::addCreatingSetsTransform(
const Block & res_header,
SetAndKeyPtr set_and_key,
StoragePtr external_table,
const SizeLimits & limits,
PreparedSetsCachePtr prepared_sets_cache)
{
resize(1);
auto transform = std::make_shared<CreatingSetsTransform>(
getHeader(),
res_header,
std::move(subquery_for_set),
std::move(set_and_key),
std::move(external_table),
limits,
context);
std::move(prepared_sets_cache));
InputPort * totals_port = nullptr;

View File

@ -33,6 +33,12 @@ class TableJoin;
class QueryPipelineBuilder;
using QueryPipelineBuilderPtr = std::unique_ptr<QueryPipelineBuilder>;
struct SetAndKey;
using SetAndKeyPtr = std::shared_ptr<SetAndKey>;
class PreparedSetsCache;
using PreparedSetsCachePtr = std::shared_ptr<PreparedSetsCache>;
class QueryPipelineBuilder
{
public:
@ -138,7 +144,12 @@ public:
/// This is used for CreatingSets.
void addPipelineBefore(QueryPipelineBuilder pipeline);
void addCreatingSetsTransform(const Block & res_header, SubqueryForSet subquery_for_set, const SizeLimits & limits, ContextPtr context);
void addCreatingSetsTransform(
const Block & res_header,
SetAndKeyPtr set_and_key,
StoragePtr external_table,
const SizeLimits & limits,
PreparedSetsCachePtr prepared_sets_cache);
PipelineExecutorPtr execute();
@ -174,6 +185,7 @@ public:
void addResources(QueryPlanResourceHolder resources_) { resources = std::move(resources_); }
void setQueryIdHolder(std::shared_ptr<QueryIdHolder> query_id_holder) { resources.query_id_holders.emplace_back(std::move(query_id_holder)); }
void addContext(ContextPtr context) { resources.interpreter_context.emplace_back(std::move(context)); }
/// Convert query pipeline to pipe.
static Pipe getPipe(QueryPipelineBuilder pipeline, QueryPlanResourceHolder & resources);

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