Merge branch 'master' of github.com:yandex/ClickHouse

This commit is contained in:
BayoNet 2019-09-05 10:23:48 +03:00
commit 595784d21f
167 changed files with 3287 additions and 1301 deletions

View File

@ -410,6 +410,6 @@ if (ENABLE_TESTS AND USE_GTEST)
# gtest framework has substandard code
target_compile_options(unit_tests_dbms PRIVATE -Wno-zero-as-null-pointer-constant -Wno-undef -Wno-sign-compare -Wno-used-but-marked-unused -Wno-missing-noreturn)
target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_parsers dbms clickhouse_common_zookeeper)
target_link_libraries(unit_tests_dbms PRIVATE ${GTEST_BOTH_LIBRARIES} clickhouse_functions clickhouse_parsers dbms clickhouse_common_zookeeper string_utils)
add_check(unit_tests_dbms)
endif ()

View File

@ -608,6 +608,7 @@ private:
if (!ends_with_backslash && (ends_with_semicolon || has_vertical_output_suffix || (!config().has("multiline") && !hasDataInSTDIN())))
{
// TODO: should we do sensitive data masking on client too? History file can be source of secret leaks.
if (input != prev_input)
{
/// Replace line breaks with spaces to prevent the following problem.
@ -1027,13 +1028,17 @@ private:
while (true)
{
Block block = async_block_input->read();
connection->sendData(block);
processed_rows += block.rows();
/// Check if server send Log packet
receiveLogs();
/// Check if server send Exception packet
auto packet_type = connection->checkPacket();
if (packet_type && *packet_type == Protocol::Server::Log)
receiveAndProcessPacket();
if (packet_type && *packet_type == Protocol::Server::Exception)
return;
connection->sendData(block);
processed_rows += block.rows();
if (!block)
break;
@ -1250,6 +1255,17 @@ private:
}
}
/// Process Log packets, used when inserting data by blocks
void receiveLogs()
{
auto packet_type = connection->checkPacket();
while (packet_type && *packet_type == Protocol::Server::Log)
{
receiveAndProcessPacket();
packet_type = connection->checkPacket();
}
}
void initBlockOutputStream(const Block & block)
{

View File

@ -74,6 +74,7 @@ void LocalServer::initialize(Poco::Util::Application & self)
if (config().has("logger") || config().has("logger.level") || config().has("logger.log"))
{
// sensitive data rules are not used here
buildLoggers(config(), logger());
}
else

View File

@ -124,6 +124,7 @@ void ODBCBridge::initialize(Application & self)
config().setString("logger", "ODBCBridge");
buildLoggers(config(), logger());
log = &logger();
hostname = config().getString("listen-host", "localhost");
port = config().getUInt("http-port");
@ -162,6 +163,12 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
context = std::make_shared<Context>(Context::createGlobal());
context->makeGlobalContext();
if (config().has("query_masking_rules"))
{
context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
setLoggerSensitiveDataMasker(logger(), context->getSensitiveDataMasker());
}
auto server = Poco::Net::HTTPServer(
new HandlerFactory("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context), server_pool, socket, http_params);
server.start();

View File

@ -278,7 +278,11 @@ int Server::main(const std::vector<std::string> & /*args*/)
* table engines could use Context on destroy.
*/
LOG_INFO(log, "Shutting down storages.");
// global_context is the owner of sensitive_data_masker, which will be destoyed after global_context->shutdown() call
setLoggerSensitiveDataMasker(logger(), nullptr);
global_context->shutdown();
LOG_DEBUG(log, "Shutted down storages.");
/** Explicitly destroy Context. It is more convenient than in destructor of Server, because logger is still available.
@ -407,6 +411,12 @@ int Server::main(const std::vector<std::string> & /*args*/)
/// Initialize main config reloader.
std::string include_from_path = config().getString("include_from", "/etc/metrika.xml");
if (config().has("query_masking_rules"))
{
global_context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto main_config_reloader = std::make_unique<ConfigReloader>(config_path,
include_from_path,
config().getString("path", ""),
@ -416,6 +426,10 @@ int Server::main(const std::vector<std::string> & /*args*/)
{
setTextLog(global_context->getTextLog());
buildLoggers(*config, logger());
if (auto masker = global_context->getSensitiveDataMasker())
{
setLoggerSensitiveDataMasker(logger(), masker);
}
global_context->setClustersConfig(config);
global_context->setMacros(std::make_unique<Macros>(*config, "macros"));
},

View File

@ -633,6 +633,13 @@ void TCPHandler::processTablesStatusRequest()
response.write(*out, client_revision);
}
void TCPHandler::receiveUnexpectedTablesStatusRequest()
{
TablesStatusRequest skip_request;
skip_request.read(*in, client_revision);
throw NetException("Unexpected packet TablesStatusRequest received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}
void TCPHandler::sendProfileInfo(const BlockStreamProfileInfo & info)
{
@ -722,6 +729,23 @@ void TCPHandler::receiveHello()
}
void TCPHandler::receiveUnexpectedHello()
{
UInt64 skip_uint_64;
String skip_string;
readStringBinary(skip_string, *in);
readVarUInt(skip_uint_64, *in);
readVarUInt(skip_uint_64, *in);
readVarUInt(skip_uint_64, *in);
readStringBinary(skip_string, *in);
readStringBinary(skip_string, *in);
readStringBinary(skip_string, *in);
throw NetException("Unexpected packet Hello received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}
void TCPHandler::sendHello()
{
writeVarUInt(Protocol::Server::Hello, *out);
@ -744,19 +768,19 @@ bool TCPHandler::receivePacket()
UInt64 packet_type = 0;
readVarUInt(packet_type, *in);
// std::cerr << "Packet: " << packet_type << std::endl;
// std::cerr << "Server got packet: " << Protocol::Client::toString(packet_type) << "\n";
switch (packet_type)
{
case Protocol::Client::Query:
if (!state.empty())
throw NetException("Unexpected packet Query received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
receiveUnexpectedQuery();
receiveQuery();
return true;
case Protocol::Client::Data:
if (state.empty())
throw NetException("Unexpected packet Data received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
receiveUnexpectedData();
return receiveData();
case Protocol::Client::Ping:
@ -768,12 +792,11 @@ bool TCPHandler::receivePacket()
return false;
case Protocol::Client::Hello:
throw Exception("Unexpected packet " + String(Protocol::Client::toString(packet_type)) + " received from client",
ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
receiveUnexpectedHello();
case Protocol::Client::TablesStatusRequest:
if (!state.empty())
throw NetException("Unexpected packet TablesStatusRequest received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
receiveUnexpectedTablesStatusRequest();
processTablesStatusRequest();
out->next();
return false;
@ -842,6 +865,26 @@ void TCPHandler::receiveQuery()
readStringBinary(state.query, *in);
}
void TCPHandler::receiveUnexpectedQuery()
{
UInt64 skip_uint_64;
String skip_string;
readStringBinary(skip_string, *in);
ClientInfo & skip_client_info = query_context->getClientInfo();
if (client_revision >= DBMS_MIN_REVISION_WITH_CLIENT_INFO)
skip_client_info.read(*in, client_revision);
Settings & skip_settings = query_context->getSettingsRef();
skip_settings.deserialize(*in);
readVarUInt(skip_uint_64, *in);
readVarUInt(skip_uint_64, *in);
readStringBinary(skip_string, *in);
throw NetException("Unexpected packet Query received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}
bool TCPHandler::receiveData()
{
@ -880,6 +923,27 @@ bool TCPHandler::receiveData()
return false;
}
void TCPHandler::receiveUnexpectedData()
{
String skip_external_table_name;
readStringBinary(skip_external_table_name, *in);
std::shared_ptr<ReadBuffer> maybe_compressed_in;
if (last_block_in.compression == Protocol::Compression::Enable)
maybe_compressed_in = std::make_shared<CompressedReadBuffer>(*in);
else
maybe_compressed_in = in;
auto skip_block_in = std::make_shared<NativeBlockInputStream>(
*maybe_compressed_in,
last_block_in.header,
client_revision,
!connection_context.getSettingsRef().low_cardinality_allow_in_native_format);
Block skip_block = skip_block_in->read();
throw NetException("Unexpected packet Data received from client", ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}
void TCPHandler::initBlockInput()
{
@ -894,6 +958,9 @@ void TCPHandler::initBlockInput()
if (state.io.out)
header = state.io.out->getHeader();
last_block_in.header = header;
last_block_in.compression = state.compression;
state.block_in = std::make_shared<NativeBlockInputStream>(
*state.maybe_compressed_in,
header,

View File

@ -82,6 +82,13 @@ struct QueryState
};
struct LastBlockInputParameters
{
Protocol::Compression compression = Protocol::Compression::Disable;
Block header;
};
class TCPHandler : public Poco::Net::TCPServerConnection
{
public:
@ -126,6 +133,9 @@ private:
/// At the moment, only one ongoing query in the connection is supported at a time.
QueryState state;
/// Last block input parameters are saved to be able to receive unexpected data packet sent after exception.
LastBlockInputParameters last_block_in;
CurrentMetrics::Increment metric_increment{CurrentMetrics::TCPConnection};
/// It is the name of the server that will be sent to the client.
@ -139,6 +149,11 @@ private:
bool receiveData();
void readData(const Settings & global_settings);
[[noreturn]] void receiveUnexpectedData();
[[noreturn]] void receiveUnexpectedQuery();
[[noreturn]] void receiveUnexpectedHello();
[[noreturn]] void receiveUnexpectedTablesStatusRequest();
/// Process INSERT query
void processInsertQuery(const Settings & global_settings);

View File

@ -0,0 +1,19 @@
<?xml version="1.0"?>
<!-- Config for test server -->
<yandex>
<query_masking_rules>
<!-- An example of query masking rule to remove profanity.
This is an example for developers. Practical examples will include something like masking of SSN or credit card numbers.
-->
<rule>
<name>profanity</name>
<regexp>(?i:shit)</regexp>
<replace>substance</replace>
</rule>
<!-- This rule is used in tests. -->
<rule>
<regexp>TOPSECRET.TOPSECRET</regexp>
<replace>[hidden]</replace>
</rule>
</query_masking_rules>
</yandex>

View File

@ -439,6 +439,20 @@
-->
<format_schema_path>/var/lib/clickhouse/format_schemas/</format_schema_path>
<!-- Uncomment to use query masking rules.
name - name for the rule (optional)
regexp - RE2 compatible regular expression (mandatory)
replace - substitution string for sensitive data (optional, by default - six asterisks)
<query_masking_rules>
<rule>
<name>hide SSN</name>
<regexp>(^|\D)\d{3}-\d{2}-\d{4}($|\D)</regexp>
<replace>000-00-0000</replace>
</rule>
</query_masking_rules>
-->
<!-- Uncomment to disable ClickHouse internal DNS caching. -->
<!-- <disable_internal_dns_cache>1</disable_internal_dns_cache> -->
</yandex>

View File

@ -589,7 +589,7 @@ Connection::Packet Connection::receivePacket()
}
//LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type));
//std::cerr << "Client got packet: " << Protocol::Server::toString(res.type) << "\n";
switch (res.type)
{
case Protocol::Server::Data: [[fallthrough]];

View File

@ -3,8 +3,6 @@
#include <string.h>
#ifdef NDEBUG
/// If set to 1 - randomize memory mappings manually (address space layout randomization) to reproduce more memory stomping bugs.
/// Note that Linux doesn't do it by default. This may lead to worse TLB performance.
#define ALLOCATOR_ASLR 0
#else
#define ALLOCATOR_ASLR 1
@ -38,23 +36,27 @@
#define MAP_ANONYMOUS MAP_ANON
#endif
/** Many modern allocators (for example, tcmalloc) do not do a mremap for realloc,
* even in case of large enough chunks of memory.
* Although this allows you to increase performance and reduce memory consumption during realloc.
/**
* Many modern allocators (for example, tcmalloc) do not do a mremap for
* realloc, even in case of large enough chunks of memory. Although this allows
* you to increase performance and reduce memory consumption during realloc.
* To fix this, we do mremap manually if the chunk of memory is large enough.
* The threshold (64 MB) is chosen quite large, since changing the address space is
* very slow, especially in the case of a large number of threads.
* We expect that the set of operations mmap/something to do/mremap can only be performed about 1000 times per second.
* The threshold (64 MB) is chosen quite large, since changing the address
* space is very slow, especially in the case of a large number of threads. We
* expect that the set of operations mmap/something to do/mremap can only be
* performed about 1000 times per second.
*
* PS. This is also required, because tcmalloc can not allocate a chunk of memory greater than 16 GB.
* P.S. This is also required, because tcmalloc can not allocate a chunk of
* memory greater than 16 GB.
*/
#ifdef NDEBUG
static constexpr size_t MMAP_THRESHOLD = 64 * (1ULL << 20);
#else
/// In debug build, use small mmap threshold to reproduce more memory stomping bugs.
/// Along with ASLR it will hopefully detect more issues than ASan.
/// The program may fail due to the limit on number of memory mappings.
/**
* In debug build, use small mmap threshold to reproduce more memory
* stomping bugs. Along with ASLR it will hopefully detect more issues than
* ASan. The program may fail due to the limit on number of memory mappings.
*/
static constexpr size_t MMAP_THRESHOLD = 4096;
#endif
@ -72,25 +74,6 @@ namespace ErrorCodes
}
}
namespace AllocatorHints
{
struct DefaultHint
{
void * mmap_hint()
{
return nullptr;
}
};
struct RandomHint
{
void * mmap_hint()
{
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(thread_local_rng));
}
};
}
/** Responsible for allocating / freeing memory. Used, for example, in PODArray, Arena.
* Also used in hash tables.
* The interface is different from std::allocator
@ -98,16 +81,12 @@ struct RandomHint
* - passing the size into the `free` method;
* - by the presence of the `alignment` argument;
* - the possibility of zeroing memory (used in hash tables);
* - hint class for mmap
* - random hint address for mmap
* - mmap_threshold for using mmap less or more
*/
template <bool clear_memory_, typename Hint, size_t mmap_threshold>
class AllocatorWithHint : Hint
template <bool clear_memory_, bool mmap_populate = false>
class Allocator
{
protected:
static constexpr bool clear_memory = clear_memory_;
static constexpr size_t small_memory_threshold = mmap_threshold;
public:
/// Allocate memory range.
void * alloc(size_t size, size_t alignment = 0)
@ -134,7 +113,8 @@ public:
/// nothing to do.
/// BTW, it's not possible to change alignment while doing realloc.
}
else if (old_size < mmap_threshold && new_size < mmap_threshold && alignment <= MALLOC_MIN_ALIGNMENT)
else if (old_size < MMAP_THRESHOLD && new_size < MMAP_THRESHOLD
&& alignment <= MALLOC_MIN_ALIGNMENT)
{
/// Resize malloc'd memory region with no special alignment requirement.
CurrentMemoryTracker::realloc(old_size, new_size);
@ -148,19 +128,20 @@ public:
if (new_size > old_size)
memset(reinterpret_cast<char *>(buf) + old_size, 0, new_size - old_size);
}
else if (old_size >= mmap_threshold && new_size >= mmap_threshold)
else if (old_size >= MMAP_THRESHOLD && new_size >= MMAP_THRESHOLD)
{
/// Resize mmap'd memory region.
CurrentMemoryTracker::realloc(old_size, new_size);
// On apple and freebsd self-implemented mremap used (common/mremap.h)
buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
buf = clickhouse_mremap(buf, old_size, new_size, MREMAP_MAYMOVE,
PROT_READ | PROT_WRITE, mmap_flags, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mremap memory chunk from " + formatReadableSizeWithBinarySuffix(old_size) + " to " + formatReadableSizeWithBinarySuffix(new_size) + ".", DB::ErrorCodes::CANNOT_MREMAP);
/// No need for zero-fill, because mmap guarantees it.
}
else if (new_size < small_memory_threshold)
else if (new_size < MMAP_THRESHOLD)
{
/// Small allocs that requires a copy. Assume there's enough memory in system. Call CurrentMemoryTracker once.
CurrentMemoryTracker::realloc(old_size, new_size);
@ -189,18 +170,30 @@ protected:
return 0;
}
static constexpr bool clear_memory = clear_memory_;
// Freshly mmapped pages are copy-on-write references to a global zero page.
// On the first write, a page fault occurs, and an actual writable page is
// allocated. If we are going to use this memory soon, such as when resizing
// hash tables, it makes sense to pre-fault the pages by passing
// MAP_POPULATE to mmap(). This takes some time, but should be faster
// overall than having a hot loop interrupted by page faults.
static constexpr int mmap_flags = MAP_PRIVATE | MAP_ANONYMOUS
| (mmap_populate ? MAP_POPULATE : 0);
private:
void * allocNoTrack(size_t size, size_t alignment)
{
void * buf;
if (size >= mmap_threshold)
if (size >= MMAP_THRESHOLD)
{
if (alignment > MMAP_MIN_ALIGNMENT)
throw DB::Exception("Too large alignment " + formatReadableSizeWithBinarySuffix(alignment) + ": more than page size when allocating "
+ formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::BAD_ARGUMENTS);
buf = mmap(Hint::mmap_hint(), size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0);
buf = mmap(getMmapHint(), size, PROT_READ | PROT_WRITE,
mmap_flags, -1, 0);
if (MAP_FAILED == buf)
DB::throwFromErrno("Allocator: Cannot mmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY);
@ -235,7 +228,7 @@ private:
void freeNoTrack(void * buf, size_t size)
{
if (size >= mmap_threshold)
if (size >= MMAP_THRESHOLD)
{
if (0 != munmap(buf, size))
DB::throwFromErrno("Allocator: Cannot munmap " + formatReadableSizeWithBinarySuffix(size) + ".", DB::ErrorCodes::CANNOT_MUNMAP);
@ -245,15 +238,22 @@ private:
::free(buf);
}
}
};
#if ALLOCATOR_ASLR
template <bool clear_memory>
using Allocator = AllocatorWithHint<clear_memory, AllocatorHints::RandomHint, MMAP_THRESHOLD>;
#ifndef NDEBUG
/// In debug builds, request mmap() at random addresses (a kind of ASLR), to
/// reproduce more memory stomping bugs. Note that Linux doesn't do it by
/// default. This may lead to worse TLB performance.
void * getMmapHint()
{
return reinterpret_cast<void *>(std::uniform_int_distribution<intptr_t>(0x100000000000UL, 0x700000000000UL)(thread_local_rng));
}
#else
template <bool clear_memory>
using Allocator = AllocatorWithHint<clear_memory, AllocatorHints::DefaultHint, MMAP_THRESHOLD>;
void * getMmapHint()
{
return nullptr;
}
#endif
};
/** When using AllocatorWithStackMemory, located on the stack,
* GCC 4.9 mistakenly assumes that we can call `free` from a pointer to the stack.

View File

@ -130,7 +130,7 @@ String FieldVisitorToString::operator() (const DecimalField<Decimal128> & x) con
String FieldVisitorToString::operator() (const UInt128 & x) const { return formatQuoted(UUID(x)); }
String FieldVisitorToString::operator() (const AggregateFunctionStateData & x) const
{
return "(" + formatQuoted(x.name) + ")" + formatQuoted(x.data);
return formatQuoted(x.data);
}
String FieldVisitorToString::operator() (const Array & x) const

View File

@ -3,7 +3,12 @@
#include <Common/Allocator.h>
using HashTableAllocator = Allocator<true>;
/**
* We are going to use the entire memory we allocated when resizing a hash
* table, so it makes sense to pre-fault the pages so that page faults don't
* interrupt the resize loop. Set the allocator parameter accordingly.
*/
using HashTableAllocator = Allocator<true /* clear_memory */, true /* mmap_populate */>;
template <size_t N = 64>
using HashTableAllocatorWithStackMemory = AllocatorWithStackMemory<HashTableAllocator, N>;

View File

@ -46,6 +46,8 @@
M(NetworkSendElapsedMicroseconds, "") \
M(ThrottlerSleepMicroseconds, "Total time a query was sleeping to conform the 'max_network_bandwidth' setting.") \
\
M(QueryMaskingRulesMatch, "Number of times query masking rules was successfully matched.") \
\
M(ReplicatedPartFetches, "Number of times a data part was downloaded from replica of a ReplicatedMergeTree table.") \
M(ReplicatedPartFailedFetches, "") \
M(ObsoleteReplicatedParts, "") \

View File

@ -0,0 +1,164 @@
#include "SensitiveDataMasker.h"
#include <set>
#include <string>
#include <atomic>
#include <re2/re2.h>
#include <re2/stringpiece.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h>
#include <Common/Exception.h>
#include <Common/StringUtils/StringUtils.h>
#ifndef NDEBUG
# include <iostream>
#endif
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
class SensitiveDataMasker::MaskingRule
{
private:
const std::string name;
const std::string replacement_string;
const std::string regexp_string;
const RE2 regexp;
const re2::StringPiece replacement;
mutable std::atomic<std::uint64_t> matches_count = 0;
public:
//* TODO: option with hyperscan? https://software.intel.com/en-us/articles/why-and-how-to-replace-pcre-with-hyperscan
// re2::set should also work quite fast, but it doesn't return the match position, only which regexp was matched
MaskingRule(const std::string & name_, const std::string & regexp_string_, const std::string & replacement_string_)
: name(name_)
, replacement_string(replacement_string_)
, regexp_string(regexp_string_)
, regexp(regexp_string, RE2::Quiet)
, replacement(replacement_string)
{
if (!regexp.ok())
throw DB::Exception(
"SensitiveDataMasker: cannot compile re2: " + regexp_string_ + ", error: " + regexp.error()
+ ". Look at https://github.com/google/re2/wiki/Syntax for reference.",
DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
uint64_t apply(std::string & data) const
{
auto m = RE2::GlobalReplace(&data, regexp, replacement);
matches_count += m;
return m;
}
const std::string & getName() const { return name; }
const std::string & getReplacementString() const { return replacement_string; }
uint64_t getMatchesCount() const { return matches_count; }
};
SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
Logger * logger = &Logger::get("SensitiveDataMaskerConfigRead");
std::set<std::string> used_names;
for (const auto & rule : keys)
{
if (startsWith(rule, "rule"))
{
auto rule_config_prefix = config_prefix + "." + rule;
auto rule_name = config.getString(rule_config_prefix + ".name", rule_config_prefix);
if (!used_names.insert(rule_name).second)
{
throw Exception(
"query_masking_rules configuration contains more than one rule named '" + rule_name + "'.",
ErrorCodes::INVALID_CONFIG_PARAMETER);
}
auto regexp = config.getString(rule_config_prefix + ".regexp", "");
if (regexp.empty())
{
throw Exception(
"query_masking_rules configuration, rule '" + rule_name + "' has no <regexp> node or <regexp> is empty.",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
auto replace = config.getString(rule_config_prefix + ".replace", "******");
try
{
addMaskingRule(rule_name, regexp, replace);
}
catch (DB::Exception & e)
{
e.addMessage("while adding query masking rule '" + rule_name + "'.");
throw;
}
}
else
{
LOG_WARNING(logger, "Unused param " << config_prefix << '.' << rule);
}
}
auto rules_count = rulesCount();
if (rules_count > 0)
{
LOG_INFO(logger, rules_count << " query masking rules loaded.");
}
}
SensitiveDataMasker::~SensitiveDataMasker() {}
void SensitiveDataMasker::addMaskingRule(
const std::string & name, const std::string & regexp_string, const std::string & replacement_string)
{
all_masking_rules.push_back(std::make_unique<MaskingRule>(name, regexp_string, replacement_string));
}
size_t SensitiveDataMasker::wipeSensitiveData(std::string & data) const
{
size_t matches = 0;
for (auto & rule : all_masking_rules)
matches += rule->apply(data);
return matches;
}
#ifndef NDEBUG
void SensitiveDataMasker::printStats()
{
for (auto & rule : all_masking_rules)
{
std::cout << rule->getName() << " (replacement to " << rule->getReplacementString() << ") matched " << rule->getMatchesCount()
<< " times" << std::endl;
}
}
#endif
size_t SensitiveDataMasker::rulesCount() const
{
return all_masking_rules.size();
}
}

View File

@ -0,0 +1,41 @@
#pragma once
#include <memory>
#include <vector>
#include <cstdint>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
}
namespace DB
{
class SensitiveDataMasker
{
private:
class MaskingRule;
std::vector<std::unique_ptr<MaskingRule>> all_masking_rules;
public:
SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
~SensitiveDataMasker();
/// Returns the number of matched rules.
size_t wipeSensitiveData(std::string & data) const;
/// Used in tests.
void addMaskingRule(const std::string & name, const std::string & regexp_string, const std::string & replacement_string);
#ifndef NDEBUG
void printStats();
#endif
size_t rulesCount() const;
};
};

View File

@ -0,0 +1,225 @@
#include <Common/Exception.h>
#include <Common/SensitiveDataMasker.h>
#include <Poco/AutoPtr.h>
#include <Poco/Util/XMLConfiguration.h>
#include <Poco/XML/XMLException.h>
#pragma GCC diagnostic ignored "-Wsign-compare"
#ifdef __clang__
# pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant"
# pragma clang diagnostic ignored "-Wundef"
#endif
#include <gtest/gtest.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
};
TEST(Common, SensitiveDataMasker)
{
Poco::AutoPtr<Poco::Util::XMLConfiguration> empty_xml_config = new Poco::Util::XMLConfiguration();
DB::SensitiveDataMasker masker(*empty_xml_config , "");
masker.addMaskingRule("all a letters", "a+", "--a--");
masker.addMaskingRule("all b letters", "b+", "--b--");
masker.addMaskingRule("all d letters", "d+", "--d--");
masker.addMaskingRule("all x letters", "x+", "--x--");
masker.addMaskingRule("rule \"d\" result", "--d--", "*****"); // RE2 regexps are applied one-by-one in order
std::string x = "aaaaaaaaaaaaa bbbbbbbbbb cccc aaaaaaaaaaaa d ";
EXPECT_EQ(masker.wipeSensitiveData(x), 5);
EXPECT_EQ(x, "--a-- --b-- cccc --a-- ***** ");
#ifndef NDEBUG
masker.printStats();
#endif
EXPECT_EQ(masker.wipeSensitiveData(x), 3);
EXPECT_EQ(x, "----a---- ----b---- cccc ----a---- ***** ");
#ifndef NDEBUG
masker.printStats();
#endif
DB::SensitiveDataMasker masker2(*empty_xml_config , "");
masker2.addMaskingRule("hide root password", "qwerty123", "******");
masker2.addMaskingRule("hide SSN", "[0-9]{3}-[0-9]{2}-[0-9]{4}", "000-00-0000");
masker2.addMaskingRule("hide email", "[A-Za-z0-9._%+-]+@[A-Za-z0-9.-]+\\.[A-Za-z]{2,4}", "hidden@hidden.test");
std::string query = "SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', 'qwerty123') WHERE ssn='123-45-6789' or "
"email='JonhSmith@secret.domain.test'";
EXPECT_EQ(masker2.wipeSensitiveData(query), 3);
EXPECT_EQ(
query,
"SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', '******') WHERE "
"ssn='000-00-0000' or email='hidden@hidden.test'");
#ifndef NDEBUG
// simple benchmark
auto start = std::chrono::high_resolution_clock::now();
constexpr unsigned long int iterations = 200000;
for (int i = 0; i < iterations; ++i)
{
std::string query2 = "SELECT id FROM mysql('localhost:3308', 'database', 'table', 'root', 'qwerty123') WHERE ssn='123-45-6789' or "
"email='JonhSmith@secret.domain.test'";
masker2.wipeSensitiveData(query2);
}
auto finish = std::chrono::high_resolution_clock::now();
std::chrono::duration<double> elapsed = finish - start;
std::cout << "Elapsed time: " << elapsed.count() << "s per " << iterations <<" calls (" << elapsed.count() * 1000000 / iterations << "µs per call)"
<< std::endl;
// I have: "Elapsed time: 3.44022s per 200000 calls (17.2011µs per call)"
masker2.printStats();
#endif
DB::SensitiveDataMasker maskerbad(*empty_xml_config , "");
// gtest has not good way to check exception content, so just do it manually (see https://github.com/google/googletest/issues/952 )
try
{
maskerbad.addMaskingRule("bad regexp", "**", "");
ADD_FAILURE() << "addMaskingRule() should throw an error" << std::endl;
}
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
"SensitiveDataMasker: cannot compile re2: **, error: no argument for repetition operator: *. Look at "
"https://github.com/google/re2/wiki/Syntax for reference.");
EXPECT_EQ(e.code(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
/* catch (...) { // not needed, gtest will react unhandled exception
FAIL() << "ERROR: Unexpected exception thrown: " << std::current_exception << std::endl; // std::current_exception is part of C++11x
} */
EXPECT_EQ(maskerbad.rulesCount(), 0);
EXPECT_EQ(maskerbad.wipeSensitiveData(x), 0);
{
std::istringstream xml_isteam(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule>
<name>hide SSN</name><!-- by default: it will use xml path, like query_masking_rules.rule[1] -->
<regexp>[0-9]{3}-[0-9]{2}-[0-9]{4}</regexp><!-- mandatory -->
<replace>000-00-0000</replace><!-- by default - six asterisks (******) -->
</rule>
<rule>
<name>hide root password</name>
<regexp>qwerty123</regexp>
</rule>
<rule>
<regexp>(?i)Ivan</regexp>
<replace>John</replace>
</rule>
<rule>
<regexp>(?i)Petrov</regexp>
<replace>Doe</replace>
</rule>
<rule>
<name>hide email</name>
<regexp>(?i)[A-Z0-9._%+-]+@[A-Z0-9.-]+\.[A-Z]{2,4}</regexp>
<replace>hidden@hidden.test</replace>
</rule>
<rule>
<name>remove selects to bad_words table</name>
<regexp>^.*bad_words.*$</regexp>
<replace>[QUERY IS CENSORED]</replace>
</rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam);
DB::SensitiveDataMasker masker_xml_based(*xml_config, "query_masking_rules");
std::string top_secret = "The e-mail of IVAN PETROV is kotik1902@sdsdf.test, and the password is qwerty123";
EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 4);
EXPECT_EQ(top_secret, "The e-mail of John Doe is hidden@hidden.test, and the password is ******");
top_secret = "SELECT * FROM bad_words";
EXPECT_EQ(masker_xml_based.wipeSensitiveData(top_secret), 1);
EXPECT_EQ(top_secret, "[QUERY IS CENSORED]");
#ifndef NDEBUG
masker_xml_based.printStats();
#endif
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule>
<name>test</name>
<regexp>abc</regexp>
</rule>
<rule>
<name>test</name>
<regexp>abc</regexp>
</rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad);
DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules");
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
"query_masking_rules configuration contains more than one rule named 'test'.");
EXPECT_EQ(e.code(), DB::ErrorCodes::INVALID_CONFIG_PARAMETER);
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule><name>test</name></rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad);
DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules");
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
"query_masking_rules configuration, rule 'test' has no <regexp> node or <regexp> is empty.");
EXPECT_EQ(e.code(), DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
<clickhouse>
<query_masking_rules>
<rule><name>test</name><regexp>())(</regexp></rule>
</query_masking_rules>
</clickhouse>)END");
Poco::AutoPtr<Poco::Util::XMLConfiguration> xml_config = new Poco::Util::XMLConfiguration(xml_isteam_bad);
DB::SensitiveDataMasker masker_xml_based_exception_check(*xml_config, "query_masking_rules");
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.message()),
"SensitiveDataMasker: cannot compile re2: ())(, error: missing ): ())(. Look at https://github.com/google/re2/wiki/Syntax for reference.: while adding query masking rule 'test'."
);
EXPECT_EQ(e.code(), DB::ErrorCodes::CANNOT_COMPILE_REGEXP);
}
}

View File

@ -59,7 +59,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingMilliseconds, connect_timeout_with_failover_ms, DBMS_DEFAULT_CONNECT_TIMEOUT_WITH_FAILOVER_MS, "Connection timeout for selecting first healthy replica.") \
M(SettingSeconds, receive_timeout, DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC, "") \
M(SettingSeconds, send_timeout, DBMS_DEFAULT_SEND_TIMEOUT_SEC, "") \
M(SettingSeconds, tcp_keep_alive_timeout, 0, "") \
M(SettingSeconds, tcp_keep_alive_timeout, 0, "The time in seconds the connection needs to remain idle before TCP starts sending keepalive probes") \
M(SettingMilliseconds, queue_max_wait_ms, 0, "The wait time in the request queue, if the number of concurrent requests exceeds the maximum.") \
M(SettingMilliseconds, connection_pool_max_wait_ms, 0, "The wait time when connection pool is full.") \
M(SettingMilliseconds, replace_running_query_max_wait_ms, 5000, "The wait time for running query with the same query_id to finish when setting 'replace_running_query' is active.") \
@ -224,6 +224,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingSeconds, http_receive_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP receive timeout") \
M(SettingBool, optimize_throw_if_noop, false, "If setting is enabled and OPTIMIZE query didn't actually assign a merge then an explanatory exception is thrown") \
M(SettingBool, use_index_for_in_with_subqueries, true, "Try using an index if there is a subquery or a table expression on the right side of the IN operator.") \
M(SettingBool, joined_subquery_requires_alias, false, "Force joined subqueries to have aliases for correct name qualification.") \
M(SettingBool, empty_result_for_aggregation_by_empty_set, false, "Return empty result when aggregating without keys on empty set.") \
M(SettingBool, allow_distributed_ddl, true, "If it is set to true, then a user is allowed to executed distributed DDL queries.") \
M(SettingUInt64, odbc_max_field_size, 1024, "Max size of filed can be read from ODBC dictionary. Long strings are truncated.") \

View File

@ -34,7 +34,7 @@ TTLBlockInputStream::TTLBlockInputStream(
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
for (const auto & [name, ttl_info] : old_ttl_infos.columns_ttl)
{
if (ttl_info.min <= current_time)
if (force || isTTLExpired(ttl_info.min))
{
new_ttl_infos.columns_ttl.emplace(name, MergeTreeDataPart::TTLInfo{});
empty_columns.emplace(name);
@ -51,7 +51,7 @@ TTLBlockInputStream::TTLBlockInputStream(
new_ttl_infos.columns_ttl.emplace(name, ttl_info);
}
if (old_ttl_infos.table_ttl.min > current_time)
if (!force && !isTTLExpired(old_ttl_infos.table_ttl.min))
new_ttl_infos.table_ttl = old_ttl_infos.table_ttl;
if (!default_expr_list->children.empty())

View File

@ -218,9 +218,11 @@ void DataTypeAggregateFunction::deserializeTextQuoted(IColumn & column, ReadBuff
}
void DataTypeAggregateFunction::deserializeWholeText(IColumn &, ReadBuffer &, const FormatSettings &) const
void DataTypeAggregateFunction::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
throw Exception("AggregateFunction data type cannot be read from text", ErrorCodes::NOT_IMPLEMENTED);
String s;
readStringUntilEOF(s, istr);
deserializeFromString(function, column, s);
}

View File

@ -348,10 +348,10 @@ private:
const UInt32 range_start = is_column_const[1] ? (*container1)[0] : (*container1)[i];
const UInt32 range_end = is_column_const[2] ? (*container2)[0] : (*container2)[i];
auto bd2 = new AggregateFunctionGroupBitmapData<T>();
bd0.rbs.rb_range(range_start, range_end, bd2->rbs);
col_to->insertFrom(reinterpret_cast<ConstAggregateDataPtr>(bd2));
col_to->insertDefault();
AggregateFunctionGroupBitmapData<T> & bd2
= *reinterpret_cast<AggregateFunctionGroupBitmapData<T> *>(col_to->getData()[i]);
bd0.rbs.rb_range(range_start, range_end, bd2.rbs);
}
block.getByPosition(result).column = std::move(col_to);
}

View File

@ -23,6 +23,7 @@
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeInterval.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <Formats/FormatSettings.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnFixedString.h>
@ -636,7 +637,7 @@ struct ConvertImplGenericFromString
{
ReadBufferFromMemory read_buffer(&chars[current_offset], offsets[i] - current_offset - 1);
data_type_to.deserializeAsTextEscaped(column_to, read_buffer, format_settings);
data_type_to.deserializeAsWholeText(column_to, read_buffer, format_settings);
if (!read_buffer.eof())
throwExceptionForIncompletelyParsedValue(read_buffer, block, result);
@ -1669,6 +1670,21 @@ private:
};
}
WrapperType createAggregateFunctionWrapper(const DataTypePtr & from_type_untyped, const DataTypeAggregateFunction * to_type) const
{
/// Conversion from String through parsing.
if (checkAndGetDataType<DataTypeString>(from_type_untyped.get()))
{
return [] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
{
ConvertImplGenericFromString::execute(block, arguments, result);
};
}
else
throw Exception{"Conversion from " + from_type_untyped->getName() + " to " + to_type->getName() +
" is not supported", ErrorCodes::CANNOT_CONVERT_TYPE};
}
WrapperType createArrayWrapper(const DataTypePtr & from_type_untyped, const DataTypeArray * to_type) const
{
/// Conversion from String through parsing.
@ -2145,13 +2161,12 @@ private:
case TypeIndex::Tuple:
return createTupleWrapper(from_type, checkAndGetDataType<DataTypeTuple>(to_type.get()));
case TypeIndex::AggregateFunction:
return createAggregateFunctionWrapper(from_type, checkAndGetDataType<DataTypeAggregateFunction>(to_type.get()));
default:
break;
}
/// It's possible to use ConvertImplGenericFromString to convert from String to AggregateFunction,
/// but it is disabled because deserializing aggregate functions state might be unsafe.
throw Exception{"Conversion from " + from_type->getName() + " to " + to_type->getName() + " is not supported",
ErrorCodes::CANNOT_CONVERT_TYPE};
}

View File

@ -1,6 +1,20 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsLogical.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <Common/FieldVisitors.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionHelpers.h>
#include <algorithm>
namespace DB
{
@ -12,4 +26,568 @@ void registerFunctionsLogical(FunctionFactory & factory)
factory.registerFunction<FunctionNot>();
}
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_COLUMN;
}
namespace
{
using namespace FunctionsLogicalDetail;
using UInt8Container = ColumnUInt8::Container;
using UInt8ColumnPtrs = std::vector<const ColumnUInt8 *>;
MutableColumnPtr convertFromTernaryData(const UInt8Container & ternary_data, const bool make_nullable)
{
const size_t rows_count = ternary_data.size();
auto new_column = ColumnUInt8::create(rows_count);
std::transform(
ternary_data.cbegin(), ternary_data.cend(), new_column->getData().begin(),
[](const auto x) { return x == Ternary::True; });
if (!make_nullable)
return new_column;
auto null_column = ColumnUInt8::create(rows_count);
std::transform(
ternary_data.cbegin(), ternary_data.cend(), null_column->getData().begin(),
[](const auto x) { return x == Ternary::Null; });
return ColumnNullable::create(std::move(new_column), std::move(null_column));
}
template <typename T>
bool tryConvertColumnToUInt8(const IColumn * column, UInt8Container & res)
{
const auto col = checkAndGetColumn<ColumnVector<T>>(column);
if (!col)
return false;
std::transform(
col->getData().cbegin(), col->getData().cend(), res.begin(),
[](const auto x) { return x != 0; });
return true;
}
void convertColumnToUInt8(const IColumn * column, UInt8Container & res)
{
if (!tryConvertColumnToUInt8<Int8>(column, res) &&
!tryConvertColumnToUInt8<Int16>(column, res) &&
!tryConvertColumnToUInt8<Int32>(column, res) &&
!tryConvertColumnToUInt8<Int64>(column, res) &&
!tryConvertColumnToUInt8<UInt16>(column, res) &&
!tryConvertColumnToUInt8<UInt32>(column, res) &&
!tryConvertColumnToUInt8<UInt64>(column, res) &&
!tryConvertColumnToUInt8<Float32>(column, res) &&
!tryConvertColumnToUInt8<Float64>(column, res))
throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
template <class Op, typename Func>
static bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res, Func && func)
{
bool has_res = false;
for (int i = static_cast<int>(in.size()) - 1; i >= 0; --i)
{
if (!isColumnConst(*in[i]))
continue;
UInt8 x = func((*in[i])[0]);
if (has_res)
{
res = Op::apply(res, x);
}
else
{
res = x;
has_res = true;
}
in.erase(in.begin() + i);
}
return has_res;
}
template <class Op>
inline bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res)
{
return extractConstColumns<Op>(
in, res,
[](const Field & value)
{
return !value.isNull() && applyVisitor(FieldVisitorConvertToNumber<bool>(), value);
}
);
}
template <class Op>
inline bool extractConstColumnsTernary(ColumnRawPtrs & in, UInt8 & res_3v)
{
return extractConstColumns<Op>(
in, res_3v,
[](const Field & value)
{
return value.isNull()
? Ternary::makeValue(false, true)
: Ternary::makeValue(applyVisitor(FieldVisitorConvertToNumber<bool>(), value));
}
);
}
template <typename Op, size_t N>
class AssociativeApplierImpl
{
using ResultValueType = typename Op::ResultType;
public:
/// Remembers the last N columns from `in`.
AssociativeApplierImpl(const UInt8ColumnPtrs & in)
: vec(in[in.size() - N]->getData()), next(in) {}
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
inline ResultValueType apply(const size_t i) const
{
const auto & a = vec[i];
if constexpr (Op::isSaturable())
return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i));
else
return Op::apply(a, next.apply(i));
}
private:
const UInt8Container & vec;
const AssociativeApplierImpl<Op, N - 1> next;
};
template <typename Op>
class AssociativeApplierImpl<Op, 1>
{
using ResultValueType = typename Op::ResultType;
public:
AssociativeApplierImpl(const UInt8ColumnPtrs & in)
: vec(in[in.size() - 1]->getData()) {}
inline ResultValueType apply(const size_t i) const { return vec[i]; }
private:
const UInt8Container & vec;
};
/// A helper class used by AssociativeGenericApplierImpl
/// Allows for on-the-fly conversion of any data type into intermediate ternary representation
using ValueGetter = std::function<Ternary::ResultType (size_t)>;
template <typename ... Types>
struct ValueGetterBuilderImpl;
template <typename Type, typename ...Types>
struct ValueGetterBuilderImpl<Type, Types...>
{
static ValueGetter build(const IColumn * x)
{
if (const auto nullable_column = typeid_cast<const ColumnNullable *>(x))
{
if (const auto nested_column = typeid_cast<const ColumnVector<Type> *>(nullable_column->getNestedColumnPtr().get()))
{
return [&null_data = nullable_column->getNullMapData(), &column_data = nested_column->getData()](size_t i)
{ return Ternary::makeValue(column_data[i], null_data[i]); };
}
else
return ValueGetterBuilderImpl<Types...>::build(x);
}
else if (const auto column = typeid_cast<const ColumnVector<Type> *>(x))
return [&column_data = column->getData()](size_t i) { return Ternary::makeValue(column_data[i]); };
else
return ValueGetterBuilderImpl<Types...>::build(x);
}
};
template <>
struct ValueGetterBuilderImpl<>
{
static ValueGetter build(const IColumn * x)
{
throw Exception(
std::string("Unknown numeric column of type: ") + demangle(typeid(x).name()),
ErrorCodes::LOGICAL_ERROR);
}
};
using ValueGetterBuilder =
ValueGetterBuilderImpl<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
/// This class together with helper class ValueGetterBuilder can be used with columns of arbitrary data type
/// Allows for on-the-fly conversion of any type of data into intermediate ternary representation
/// and eliminates the need to materialize data columns in intermediate representation
template <typename Op, size_t N>
class AssociativeGenericApplierImpl
{
using ResultValueType = typename Op::ResultType;
public:
/// Remembers the last N columns from `in`.
AssociativeGenericApplierImpl(const ColumnRawPtrs & in)
: val_getter{ValueGetterBuilder::build(in[in.size() - N])}, next{in} {}
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
inline ResultValueType apply(const size_t i) const
{
const auto a = val_getter(i);
if constexpr (Op::isSaturable())
return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i));
else
return Op::apply(a, next.apply(i));
}
private:
const ValueGetter val_getter;
const AssociativeGenericApplierImpl<Op, N - 1> next;
};
template <typename Op>
class AssociativeGenericApplierImpl<Op, 1>
{
using ResultValueType = typename Op::ResultType;
public:
/// Remembers the last N columns from `in`.
AssociativeGenericApplierImpl(const ColumnRawPtrs & in)
: val_getter{ValueGetterBuilder::build(in[in.size() - 1])} {}
inline ResultValueType apply(const size_t i) const { return val_getter(i); }
private:
const ValueGetter val_getter;
};
/// Apply target function by feeding it "batches" of N columns
/// Combining 10 columns per pass is the fastest for large block sizes.
/// For small block sizes - more columns is faster.
template <
typename Op, template <typename, size_t> typename OperationApplierImpl, size_t N = 10>
struct OperationApplier
{
template <typename Columns, typename ResultColumn>
static void apply(Columns & in, ResultColumn & result)
{
while (in.size() > 1)
{
doBatchedApply(in, result->getData());
in.push_back(result.get());
}
}
template <typename Columns, typename ResultData>
static void NO_INLINE doBatchedApply(Columns & in, ResultData & result_data)
{
if (N > in.size())
{
OperationApplier<Op, OperationApplierImpl, N - 1>::doBatchedApply(in, result_data);
return;
}
const OperationApplierImpl<Op, N> operationApplierImpl(in);
size_t i = 0;
for (auto & res : result_data)
res = operationApplierImpl.apply(i++);
in.erase(in.end() - N, in.end());
}
};
template <
typename Op, template <typename, size_t> typename OperationApplierImpl>
struct OperationApplier<Op, OperationApplierImpl, 1>
{
template <typename Columns, typename Result>
static void NO_INLINE doBatchedApply(Columns &, Result &)
{
throw Exception(
"OperationApplier<...>::apply(...): not enough arguments to run this method",
ErrorCodes::LOGICAL_ERROR);
}
};
template <class Op>
static void executeForTernaryLogicImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & result_info, size_t input_rows_count)
{
/// Combine all constant columns into a single constant value.
UInt8 const_3v_value = 0;
const bool has_consts = extractConstColumnsTernary<Op>(arguments, const_3v_value);
/// If the constant value uniquely determines the result, return it.
if (has_consts && (arguments.empty() || (Op::isSaturable() && Op::isSaturatedValue(const_3v_value))))
{
result_info.column = ColumnConst::create(
convertFromTernaryData(UInt8Container({const_3v_value}), result_info.type->isNullable()),
input_rows_count
);
return;
}
const auto result_column = ColumnUInt8::create(input_rows_count);
MutableColumnPtr const_column_holder;
if (has_consts)
{
const_column_holder =
convertFromTernaryData(UInt8Container(input_rows_count, const_3v_value), const_3v_value == Ternary::Null);
arguments.push_back(const_column_holder.get());
}
OperationApplier<Op, AssociativeGenericApplierImpl>::apply(arguments, result_column);
result_info.column = convertFromTernaryData(result_column->getData(), result_info.type->isNullable());
}
template <typename Op, typename ... Types>
struct TypedExecutorInvoker;
template <typename Op>
using FastApplierImpl =
TypedExecutorInvoker<Op, UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Float32, Float64>;
template <typename Op, typename Type, typename ... Types>
struct TypedExecutorInvoker<Op, Type, Types ...>
{
template <typename T, typename Result>
static void apply(const ColumnVector<T> & x, const IColumn & y, Result & result)
{
if (const auto column = typeid_cast<const ColumnVector<Type> *>(&y))
std::transform(
x.getData().cbegin(), x.getData().cend(),
column->getData().cbegin(), result.begin(),
[](const auto a, const auto b) { return Op::apply(!!a, !!b); });
else
TypedExecutorInvoker<Op, Types ...>::template apply<T>(x, y, result);
}
template <typename Result>
static void apply(const IColumn & x, const IColumn & y, Result & result)
{
if (const auto column = typeid_cast<const ColumnVector<Type> *>(&x))
FastApplierImpl<Op>::template apply<Type>(*column, y, result);
else
TypedExecutorInvoker<Op, Types ...>::apply(x, y, result);
}
};
template <typename Op>
struct TypedExecutorInvoker<Op>
{
template <typename T, typename Result>
static void apply(const ColumnVector<T> &, const IColumn & y, Result &)
{
throw Exception(std::string("Unknown numeric column y of type: ") + demangle(typeid(y).name()), ErrorCodes::LOGICAL_ERROR);
}
template <typename Result>
static void apply(const IColumn & x, const IColumn &, Result &)
{
throw Exception(std::string("Unknown numeric column x of type: ") + demangle(typeid(x).name()), ErrorCodes::LOGICAL_ERROR);
}
};
template <class Op>
static void basicExecuteImpl(ColumnRawPtrs arguments, ColumnWithTypeAndName & result_info, size_t input_rows_count)
{
/// Combine all constant columns into a single constant value.
UInt8 const_val = 0;
bool has_consts = extractConstColumns<Op>(arguments, const_val);
/// If the constant value uniquely determines the result, return it.
if (has_consts && (arguments.empty() || Op::apply(const_val, 0) == Op::apply(const_val, 1)))
{
if (!arguments.empty())
const_val = Op::apply(const_val, 0);
result_info.column = DataTypeUInt8().createColumnConst(input_rows_count, toField(const_val));
return;
}
/// If the constant value is a neutral element, let's forget about it.
if (has_consts && Op::apply(const_val, 0) == 0 && Op::apply(const_val, 1) == 1)
has_consts = false;
UInt8ColumnPtrs uint8_args;
auto col_res = ColumnUInt8::create();
UInt8Container & vec_res = col_res->getData();
if (has_consts)
{
vec_res.assign(input_rows_count, const_val);
uint8_args.push_back(col_res.get());
}
else
{
vec_res.resize(input_rows_count);
}
/// FastPath detection goes in here
if (arguments.size() == (has_consts ? 1 : 2))
{
if (has_consts)
FastApplierImpl<Op>::apply(*arguments[0], *col_res, col_res->getData());
else
FastApplierImpl<Op>::apply(*arguments[0], *arguments[1], col_res->getData());
result_info.column = std::move(col_res);
return;
}
/// Convert all columns to UInt8
Columns converted_columns;
for (const IColumn * column : arguments)
{
if (auto uint8_column = checkAndGetColumn<ColumnUInt8>(column))
uint8_args.push_back(uint8_column);
else
{
auto converted_column = ColumnUInt8::create(input_rows_count);
convertColumnToUInt8(column, converted_column->getData());
uint8_args.push_back(converted_column.get());
converted_columns.emplace_back(std::move(converted_column));
}
}
OperationApplier<Op, AssociativeApplierImpl>::apply(uint8_args, col_res);
/// This is possible if there is exactly one non-constant among the arguments, and it is of type UInt8.
if (uint8_args[0] != col_res.get())
vec_res.assign(uint8_args[0]->getData());
result_info.column = std::move(col_res);
}
}
template <typename Impl, typename Name>
DataTypePtr FunctionAnyArityLogical<Impl, Name>::getReturnTypeImpl(const DataTypes & arguments) const
{
if (arguments.size() < 2)
throw Exception("Number of arguments for function \"" + getName() + "\" should be at least 2: passed "
+ toString(arguments.size()),
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION);
bool has_nullable_arguments = false;
for (size_t i = 0; i < arguments.size(); ++i)
{
const auto & arg_type = arguments[i];
if (!has_nullable_arguments)
{
has_nullable_arguments = arg_type->isNullable();
if (has_nullable_arguments && !Impl::specialImplementationForNulls())
throw Exception("Logical error: Unexpected type of argument for function \"" + getName() + "\": "
" argument " + toString(i + 1) + " is of type " + arg_type->getName(), ErrorCodes::LOGICAL_ERROR);
}
if (!(isNativeNumber(arg_type)
|| (Impl::specialImplementationForNulls() && (arg_type->onlyNull() || isNativeNumber(removeNullable(arg_type))))))
throw Exception("Illegal type ("
+ arg_type->getName()
+ ") of " + toString(i + 1) + " argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
auto result_type = std::make_shared<DataTypeUInt8>();
return has_nullable_arguments
? makeNullable(result_type)
: result_type;
}
template <typename Impl, typename Name>
void FunctionAnyArityLogical<Impl, Name>::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count)
{
ColumnRawPtrs args_in;
for (const auto arg_index : arguments)
args_in.push_back(block.getByPosition(arg_index).column.get());
auto & result_info = block.getByPosition(result_index);
if (result_info.type->isNullable())
executeForTernaryLogicImpl<Impl>(std::move(args_in), result_info, input_rows_count);
else
basicExecuteImpl<Impl>(std::move(args_in), result_info, input_rows_count);
}
template <typename A, typename Op>
struct UnaryOperationImpl
{
using ResultType = typename Op::ResultType;
using ArrayA = typename ColumnVector<A>::Container;
using ArrayC = typename ColumnVector<ResultType>::Container;
static void NO_INLINE vector(const ArrayA & a, ArrayC & c)
{
std::transform(
a.cbegin(), a.cend(), c.begin(),
[](const auto x) { return Op::apply(x); });
}
};
template <template <typename> class Impl, typename Name>
DataTypePtr FunctionUnaryLogical<Impl, Name>::getReturnTypeImpl(const DataTypes & arguments) const
{
if (!isNativeNumber(arguments[0]))
throw Exception("Illegal type ("
+ arguments[0]->getName()
+ ") of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt8>();
}
template <template <typename> class Impl, typename T>
bool functionUnaryExecuteType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (auto col = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnUInt8::create();
typename ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
UnaryOperationImpl<T, Impl<T>>::vector(col->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
}
template <template <typename> class Impl, typename Name>
void FunctionUnaryLogical<Impl, Name>::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
if (!(functionUnaryExecuteType<Impl, UInt8>(block, arguments, result)
|| functionUnaryExecuteType<Impl, UInt16>(block, arguments, result)
|| functionUnaryExecuteType<Impl, UInt32>(block, arguments, result)
|| functionUnaryExecuteType<Impl, UInt64>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Int8>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Int16>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Int32>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Int64>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Float32>(block, arguments, result)
|| functionUnaryExecuteType<Impl, Float64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
}

View File

@ -1,15 +1,10 @@
#pragma once
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypeNullable.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <Core/Types.h>
#include <Core/Defines.h>
#include <DataTypes/IDataType.h>
#include <Functions/IFunction.h>
#include <Functions/FunctionHelpers.h>
#include <Common/FieldVisitors.h>
#include <IO/WriteHelpers.h>
#include <type_traits>
@ -23,84 +18,77 @@
#endif
/** Logical functions AND, OR, XOR and NOT support three-valued (or ternary) logic
* https://en.wikibooks.org/wiki/Structured_Query_Language/NULLs_and_the_Three_Valued_Logic
*
* Functions XOR and NOT rely on "default implementation for NULLs":
* - if any of the arguments is of Nullable type, the return value type is Nullable
* - if any of the arguments is NULL, the return value is NULL
*
* Functions AND and OR provide their own special implementations for ternary logic
*/
namespace DB
{
namespace ErrorCodes
namespace FunctionsLogicalDetail
{
extern const int LOGICAL_ERROR;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
namespace Ternary
{
using ResultType = UInt8;
static constexpr UInt8 False = 0;
static constexpr UInt8 True = -1;
static constexpr UInt8 Null = 1;
template <typename T>
inline ResultType makeValue(T value)
{
return value != 0 ? Ternary::True : Ternary::False;
}
template <typename T>
inline ResultType makeValue(T value, bool is_null)
{
if (is_null)
return Ternary::Null;
return makeValue<T>(value);
}
}
/** Behaviour in presence of NULLs:
*
* Functions AND, XOR, NOT use default implementation for NULLs:
* - if one of arguments is Nullable, they return Nullable result where NULLs are returned when at least one argument was NULL.
*
* But function OR is different.
* It always return non-Nullable result and NULL are equivalent to 0 (false).
* For example, 1 OR NULL returns 1, not NULL.
*/
struct AndImpl
{
static inline constexpr bool isSaturable()
{
return true;
}
using ResultType = UInt8;
static inline constexpr bool isSaturatedValue(bool a)
{
return !a;
}
static inline constexpr bool apply(bool a, bool b)
{
return a && b;
}
static inline constexpr bool specialImplementationForNulls() { return false; }
static inline constexpr bool isSaturable() { return true; }
static inline constexpr bool isSaturatedValue(UInt8 a) { return a == Ternary::False; }
static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return a & b; }
static inline constexpr bool specialImplementationForNulls() { return true; }
};
struct OrImpl
{
static inline constexpr bool isSaturable()
{
return true;
}
static inline constexpr bool isSaturatedValue(bool a)
{
return a;
}
static inline constexpr bool apply(bool a, bool b)
{
return a || b;
}
using ResultType = UInt8;
static inline constexpr bool isSaturable() { return true; }
static inline constexpr bool isSaturatedValue(UInt8 a) { return a == Ternary::True; }
static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return a | b; }
static inline constexpr bool specialImplementationForNulls() { return true; }
};
struct XorImpl
{
static inline constexpr bool isSaturable()
{
return false;
}
static inline constexpr bool isSaturatedValue(bool)
{
return false;
}
static inline constexpr bool apply(bool a, bool b)
{
return a != b;
}
using ResultType = UInt8;
static inline constexpr bool isSaturable() { return false; }
static inline constexpr bool isSaturatedValue(bool) { return false; }
/** Considering that CH uses UInt8 for representation of boolean values this function
* returns 255 as "true" but the current implementation of logical functions suggests that
* any nonzero value is "true" as well. Also the current code provides no guarantee
* for "true" to be represented with the value of 1.
*/
static inline constexpr ResultType apply(UInt8 a, UInt8 b) { return (a != b) ? Ternary::True : Ternary::False; }
static inline constexpr bool specialImplementationForNulls() { return false; }
#if USE_EMBEDDED_COMPILER
@ -116,7 +104,7 @@ struct NotImpl
{
using ResultType = UInt8;
static inline UInt8 apply(A a)
static inline ResultType apply(A a)
{
return !a;
}
@ -129,75 +117,6 @@ struct NotImpl
#endif
};
using UInt8Container = ColumnUInt8::Container;
using UInt8ColumnPtrs = std::vector<const ColumnUInt8 *>;
template <typename Op, size_t N>
struct AssociativeOperationImpl
{
/// Erases the N last columns from `in` (if there are less, then all) and puts into `result` their combination.
static void NO_INLINE execute(UInt8ColumnPtrs & in, UInt8Container & result)
{
if (N > in.size())
{
AssociativeOperationImpl<Op, N - 1>::execute(in, result);
return;
}
AssociativeOperationImpl<Op, N> operation(in);
in.erase(in.end() - N, in.end());
size_t n = result.size();
for (size_t i = 0; i < n; ++i)
{
result[i] = operation.apply(i);
}
}
const UInt8Container & vec;
AssociativeOperationImpl<Op, N - 1> continuation;
/// Remembers the last N columns from `in`.
AssociativeOperationImpl(UInt8ColumnPtrs & in)
: vec(in[in.size() - N]->getData()), continuation(in) {}
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
inline UInt8 apply(size_t i) const
{
if (Op::isSaturable())
{
UInt8 a = vec[i];
return Op::isSaturatedValue(a) ? a : continuation.apply(i);
}
else
{
return Op::apply(vec[i], continuation.apply(i));
}
}
};
template <typename Op>
struct AssociativeOperationImpl<Op, 1>
{
static void execute(UInt8ColumnPtrs &, UInt8Container &)
{
throw Exception("Logical error: AssociativeOperationImpl<Op, 1>::execute called", ErrorCodes::LOGICAL_ERROR);
}
const UInt8Container & vec;
AssociativeOperationImpl(UInt8ColumnPtrs & in)
: vec(in[in.size() - 1]->getData()) {}
inline UInt8 apply(size_t i) const
{
return vec[i];
}
};
template <typename Impl, typename Name>
class FunctionAnyArityLogical : public IFunction
{
@ -205,90 +124,6 @@ public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionAnyArityLogical>(); }
private:
bool extractConstColumns(ColumnRawPtrs & in, UInt8 & res)
{
bool has_res = false;
for (int i = static_cast<int>(in.size()) - 1; i >= 0; --i)
{
if (!isColumnConst(*in[i]))
continue;
Field value = (*in[i])[0];
UInt8 x = !value.isNull() && applyVisitor(FieldVisitorConvertToNumber<bool>(), value);
if (has_res)
{
res = Impl::apply(res, x);
}
else
{
res = x;
has_res = true;
}
in.erase(in.begin() + i);
}
return has_res;
}
template <typename T>
bool convertTypeToUInt8(const IColumn * column, UInt8Container & res)
{
auto col = checkAndGetColumn<ColumnVector<T>>(column);
if (!col)
return false;
const auto & vec = col->getData();
size_t n = res.size();
for (size_t i = 0; i < n; ++i)
res[i] = !!vec[i];
return true;
}
template <typename T>
bool convertNullableTypeToUInt8(const IColumn * column, UInt8Container & res)
{
auto col_nullable = checkAndGetColumn<ColumnNullable>(column);
auto col = checkAndGetColumn<ColumnVector<T>>(&col_nullable->getNestedColumn());
if (!col)
return false;
const auto & vec = col->getData();
const auto & null_map = col_nullable->getNullMapData();
size_t n = res.size();
for (size_t i = 0; i < n; ++i)
res[i] = !!vec[i] && !null_map[i];
return true;
}
void convertToUInt8(const IColumn * column, UInt8Container & res)
{
if (!convertTypeToUInt8<Int8>(column, res) &&
!convertTypeToUInt8<Int16>(column, res) &&
!convertTypeToUInt8<Int32>(column, res) &&
!convertTypeToUInt8<Int64>(column, res) &&
!convertTypeToUInt8<UInt16>(column, res) &&
!convertTypeToUInt8<UInt32>(column, res) &&
!convertTypeToUInt8<UInt64>(column, res) &&
!convertTypeToUInt8<Float32>(column, res) &&
!convertTypeToUInt8<Float64>(column, res) &&
!convertNullableTypeToUInt8<Int8>(column, res) &&
!convertNullableTypeToUInt8<Int16>(column, res) &&
!convertNullableTypeToUInt8<Int32>(column, res) &&
!convertNullableTypeToUInt8<Int64>(column, res) &&
!convertNullableTypeToUInt8<UInt8>(column, res) &&
!convertNullableTypeToUInt8<UInt16>(column, res) &&
!convertNullableTypeToUInt8<UInt32>(column, res) &&
!convertNullableTypeToUInt8<UInt64>(column, res) &&
!convertNullableTypeToUInt8<Float32>(column, res) &&
!convertNullableTypeToUInt8<Float64>(column, res))
throw Exception("Unexpected type of column: " + column->getName(), ErrorCodes::ILLEGAL_COLUMN);
}
public:
String getName() const override
{
@ -301,98 +136,12 @@ public:
bool useDefaultImplementationForNulls() const override { return !Impl::specialImplementationForNulls(); }
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() < 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be at least 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
for (size_t i = 0; i < arguments.size(); ++i)
if (!(isNativeNumber(arguments[i])
|| (Impl::specialImplementationForNulls() && (arguments[i]->onlyNull() || isNativeNumber(removeNullable(arguments[i]))))))
throw Exception("Illegal type ("
+ arguments[i]->getName()
+ ") of " + toString(i + 1) + " argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt8>();
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
size_t num_arguments = arguments.size();
ColumnRawPtrs in(num_arguments);
for (size_t i = 0; i < num_arguments; ++i)
in[i] = block.getByPosition(arguments[i]).column.get();
size_t rows = in[0]->size();
/// Combine all constant columns into a single value.
UInt8 const_val = 0;
bool has_consts = extractConstColumns(in, const_val);
// If this value uniquely determines the result, return it.
if (has_consts && (in.empty() || Impl::apply(const_val, 0) == Impl::apply(const_val, 1)))
{
if (!in.empty())
const_val = Impl::apply(const_val, 0);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(rows, toField(const_val));
return;
}
/// If this value is a neutral element, let's forget about it.
if (has_consts && Impl::apply(const_val, 0) == 0 && Impl::apply(const_val, 1) == 1)
has_consts = false;
auto col_res = ColumnUInt8::create();
UInt8Container & vec_res = col_res->getData();
if (has_consts)
{
vec_res.assign(rows, const_val);
in.push_back(col_res.get());
}
else
{
vec_res.resize(rows);
}
/// Convert all columns to UInt8
UInt8ColumnPtrs uint8_in;
Columns converted_columns;
for (const IColumn * column : in)
{
if (auto uint8_column = checkAndGetColumn<ColumnUInt8>(column))
uint8_in.push_back(uint8_column);
else
{
auto converted_column = ColumnUInt8::create(rows);
convertToUInt8(column, converted_column->getData());
uint8_in.push_back(converted_column.get());
converted_columns.emplace_back(std::move(converted_column));
}
}
/// Effeciently combine all the columns of the correct type.
while (uint8_in.size() > 1)
{
/// With a large block size, combining 6 columns per pass is the fastest.
/// When small - more, is faster.
AssociativeOperationImpl<Impl, 10>::execute(uint8_in, vec_res);
uint8_in.push_back(col_res.get());
}
/// This is possible if there is exactly one non-constant among the arguments, and it is of type UInt8.
if (uint8_in[0] != col_res.get())
vec_res.assign(uint8_in[0]->getData());
block.getByPosition(result).column = std::move(col_res);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result_index, size_t input_rows_count) override;
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes &) const override { return true; }
bool isCompilableImpl(const DataTypes &) const override { return useDefaultImplementationForNulls(); }
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, ValuePlaceholders values) const override
{
@ -431,27 +180,6 @@ public:
};
template <typename A, typename Op>
struct UnaryOperationImpl
{
using ResultType = typename Op::ResultType;
using ArrayA = typename ColumnVector<A>::Container;
using ArrayC = typename ColumnVector<ResultType>::Container;
static void NO_INLINE vector(const ArrayA & a, ArrayC & c)
{
size_t size = a.size();
for (size_t i = 0; i < size; ++i)
c[i] = Op::apply(a[i]);
}
static void constant(A a, ResultType & c)
{
c = Op::apply(a);
}
};
template <template <typename> class Impl, typename Name>
class FunctionUnaryLogical : public IFunction
{
@ -459,25 +187,6 @@ public:
static constexpr auto name = Name::name;
static FunctionPtr create(const Context &) { return std::make_shared<FunctionUnaryLogical>(); }
private:
template <typename T>
bool executeType(Block & block, const ColumnNumbers & arguments, size_t result)
{
if (auto col = checkAndGetColumn<ColumnVector<T>>(block.getByPosition(arguments[0]).column.get()))
{
auto col_res = ColumnUInt8::create();
typename ColumnUInt8::Container & vec_res = col_res->getData();
vec_res.resize(col->getData().size());
UnaryOperationImpl<T, Impl<T>>::vector(col->getData(), vec_res);
block.getByPosition(result).column = std::move(col_res);
return true;
}
return false;
}
public:
String getName() const override
{
@ -486,35 +195,11 @@ public:
size_t getNumberOfArguments() const override { return 1; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isNativeNumber(arguments[0]))
throw Exception("Illegal type ("
+ arguments[0]->getName()
+ ") of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt8>();
}
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override;
bool useDefaultImplementationForConstants() const override { return true; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
{
if (!(executeType<UInt8>(block, arguments, result)
|| executeType<UInt16>(block, arguments, result)
|| executeType<UInt32>(block, arguments, result)
|| executeType<UInt64>(block, arguments, result)
|| executeType<Int8>(block, arguments, result)
|| executeType<Int16>(block, arguments, result)
|| executeType<Int32>(block, arguments, result)
|| executeType<Int64>(block, arguments, result)
|| executeType<Float32>(block, arguments, result)
|| executeType<Float64>(block, arguments, result)))
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
+ " of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override;
#if USE_EMBEDDED_COMPILER
bool isCompilableImpl(const DataTypes &) const override { return true; }
@ -527,15 +212,16 @@ public:
#endif
};
}
struct NameAnd { static constexpr auto name = "and"; };
struct NameOr { static constexpr auto name = "or"; };
struct NameXor { static constexpr auto name = "xor"; };
struct NameNot { static constexpr auto name = "not"; };
using FunctionAnd = FunctionAnyArityLogical<AndImpl, NameAnd>;
using FunctionOr = FunctionAnyArityLogical<OrImpl, NameOr>;
using FunctionXor = FunctionAnyArityLogical<XorImpl, NameXor>;
using FunctionNot = FunctionUnaryLogical<NotImpl, NameNot>;
using FunctionAnd = FunctionsLogicalDetail::FunctionAnyArityLogical<FunctionsLogicalDetail::AndImpl, NameAnd>;
using FunctionOr = FunctionsLogicalDetail::FunctionAnyArityLogical<FunctionsLogicalDetail::OrImpl, NameOr>;
using FunctionXor = FunctionsLogicalDetail::FunctionAnyArityLogical<FunctionsLogicalDetail::XorImpl, NameXor>;
using FunctionNot = FunctionsLogicalDetail::FunctionUnaryLogical<FunctionsLogicalDetail::NotImpl, NameNot>;
}

View File

@ -1,5 +1,6 @@
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionsComparison.h>
#include <Functions/FunctionsLogical.h>
namespace DB

View File

@ -88,21 +88,12 @@ public:
}
/** How many bytes have been read/written, counting those that are still in the buffer. */
size_t count() const
{
return bytes + offset();
}
size_t count() const { return bytes + offset(); }
/** Check that there is more bytes in buffer after cursor. */
bool ALWAYS_INLINE hasPendingData() const
{
return pos != working_buffer.end();
}
bool ALWAYS_INLINE hasPendingData() const { return available() > 0; }
bool isPadded() const
{
return padded;
}
bool isPadded() const { return padded; }
protected:
/// Read/write position.

View File

@ -1,57 +0,0 @@
#pragma once
#include <IO/ReadBuffer.h>
#include <Common/typeid_cast.h>
namespace DB
{
/// Consistently reads from one sub-buffer in a circle, and delimits its output with a character.
/// Owns sub-buffer.
class DelimitedReadBuffer : public ReadBuffer
{
public:
DelimitedReadBuffer(std::unique_ptr<ReadBuffer> buffer_, char delimiter_) : ReadBuffer(nullptr, 0), buffer(std::move(buffer_)), delimiter(delimiter_)
{
// TODO: check that `buffer_` is not nullptr.
}
template <class BufferType>
BufferType * subBufferAs()
{
return typeid_cast<BufferType *>(buffer.get());
}
void reset()
{
BufferBase::set(nullptr, 0, 0);
}
protected:
// XXX: don't know how to guarantee that the next call to this method is done after we read all previous data.
bool nextImpl() override
{
if (put_delimiter)
{
BufferBase::set(&delimiter, 1, 0);
put_delimiter = false;
}
else
{
if (!buffer->next())
return false;
BufferBase::set(buffer->position(), buffer->available(), 0);
put_delimiter = (delimiter != 0);
}
return true;
}
private:
std::unique_ptr<ReadBuffer> buffer; // FIXME: should be `const`, but `ReadBuffer` doesn't allow
char delimiter; // FIXME: should be `const`, but `ReadBuffer` doesn't allow
bool put_delimiter = false;
};
}

View File

@ -39,7 +39,8 @@
#endif
#define DEFAULT_MAX_STRING_SIZE 0x00FFFFFFULL
/// 1 GiB
#define DEFAULT_MAX_STRING_SIZE (1ULL << 30)
namespace DB

View File

@ -209,7 +209,15 @@ bool AnalyzedJoin::sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y)
&& x->table_join.strictness == y->table_join.strictness
&& x->key_names_left == y->key_names_left
&& x->key_names_right == y->key_names_right
&& x->columns_added_by_join == y->columns_added_by_join;
&& x->columns_added_by_join == y->columns_added_by_join
&& x->hash_join == y->hash_join;
}
BlockInputStreamPtr AnalyzedJoin::createStreamWithNonJoinedDataIfFullOrRightJoin(const Block & source_header, UInt64 max_block_size) const
{
if (isRightOrFull(table_join.kind))
return hash_join->createStreamWithNonJoinedRows(source_header, *this, max_block_size);
return {};
}
JoinPtr AnalyzedJoin::makeHashJoin(const Block & sample_block, const SizeLimits & size_limits_for_join) const
@ -219,6 +227,21 @@ JoinPtr AnalyzedJoin::makeHashJoin(const Block & sample_block, const SizeLimits
return join;
}
void AnalyzedJoin::joinBlock(Block & block) const
{
hash_join->joinBlock(block, *this);
}
void AnalyzedJoin::joinTotals(Block & block) const
{
hash_join->joinTotals(block);
}
bool AnalyzedJoin::hasTotals() const
{
return hash_join->hasTotals();
}
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context)
{
NamesAndTypesList names_and_type_list;

View File

@ -4,6 +4,7 @@
#include <Core/NamesAndTypes.h>
#include <Core/SettingsCommon.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <utility>
#include <memory>
@ -19,7 +20,7 @@ class Block;
class Join;
using JoinPtr = std::shared_ptr<Join>;
struct AnalyzedJoin
class AnalyzedJoin
{
/** Query of the form `SELECT expr(x) AS k FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
* The join is made by column k.
@ -33,7 +34,6 @@ struct AnalyzedJoin
* It's possible to use name `expr(t2 columns)`.
*/
private:
friend class SyntaxAnalyzer;
Names key_names_left;
@ -53,6 +53,8 @@ private:
/// Original name -> name. Only ranamed columns.
std::unordered_map<String, String> renames;
JoinPtr hash_join;
public:
void addUsingKey(const ASTPtr & ast);
void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast);
@ -79,7 +81,12 @@ public:
const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; }
const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; }
void setHashJoin(JoinPtr join) { hash_join = join; }
JoinPtr makeHashJoin(const Block & sample_block, const SizeLimits & size_limits_for_join) const;
BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin(const Block & source_header, UInt64 max_block_size) const;
void joinBlock(Block & block) const;
void joinTotals(Block & block) const;
bool hasTotals() const;
static bool sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y);
};

View File

@ -10,7 +10,7 @@ namespace DB
{
class ASTIdentifier;
struct AnalyzedJoin;
class AnalyzedJoin;
class CollectJoinOnKeysMatcher
{

View File

@ -86,6 +86,7 @@ namespace ErrorCodes
extern const int SESSION_NOT_FOUND;
extern const int SESSION_IS_LOCKED;
extern const int CANNOT_GET_CREATE_TABLE_QUERY;
extern const int LOGICAL_ERROR;
}
@ -142,6 +143,8 @@ struct ContextShared
std::unique_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
mutable std::unique_ptr<CompressionCodecSelector> compression_codec_selector;
/// Allows to remove sensitive data from queries using set of regexp-based rules
std::unique_ptr<SensitiveDataMasker> sensitive_data_masker;
std::optional<MergeTreeSettings> merge_tree_settings; /// Settings of MergeTree* engines.
size_t max_table_size_to_drop = 50000000000lu; /// Protects MergeTree tables from accidental DROP (50GB by default)
size_t max_partition_size_to_drop = 50000000000lu; /// Protects MergeTree partitions from accidental DROP (50GB by default)
@ -284,6 +287,8 @@ struct ContextShared
/// Stop trace collector if any
trace_collector.reset();
sensitive_data_masker.reset();
}
bool hasTraceCollector()
@ -533,6 +538,23 @@ String Context::getUserFilesPath() const
return shared->user_files_path;
}
void Context::setSensitiveDataMasker(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker)
{
if (!sensitive_data_masker)
throw Exception("Logical error: the 'sensitive_data_masker' is not set", ErrorCodes::LOGICAL_ERROR);
if (sensitive_data_masker->rulesCount() > 0)
{
auto lock = getLock();
shared->sensitive_data_masker = std::move(sensitive_data_masker);
}
}
SensitiveDataMasker * Context::getSensitiveDataMasker() const
{
return shared->sensitive_data_masker.get();
}
void Context::setPath(const String & path)
{
auto lock = getLock();

View File

@ -12,6 +12,7 @@
#include <Common/MultiVersion.h>
#include <Common/ThreadPool.h>
#include "config_core.h"
#include <Common/SensitiveDataMasker.h>
#include <Storages/IStorage_fwd.h>
#include <atomic>
#include <chrono>
@ -177,6 +178,9 @@ public:
String getFlagsPath() const;
String getUserFilesPath() const;
void setSensitiveDataMasker(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker);
SensitiveDataMasker * getSensitiveDataMasker() const;
void setPath(const String & path);
void setTemporaryPath(const String & path);
void setFlagsPath(const String & path);

View File

@ -3,11 +3,12 @@
#include <Common/SipHash.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionJIT.h>
#include <Interpreters/Join.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnArray.h>
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <set>
@ -44,8 +45,8 @@ Names ExpressionAction::getNeededColumns() const
res.insert(res.end(), array_joined_columns.begin(), array_joined_columns.end());
if (join_params)
res.insert(res.end(), join_params->keyNamesLeft().begin(), join_params->keyNamesLeft().end());
if (table_join)
res.insert(res.end(), table_join->keyNamesLeft().begin(), table_join->keyNamesLeft().end());
for (const auto & column : projection)
res.push_back(column.first);
@ -159,12 +160,11 @@ ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_column
return a;
}
ExpressionAction ExpressionAction::ordinaryJoin(std::shared_ptr<AnalyzedJoin> join_params, std::shared_ptr<const Join> hash_join)
ExpressionAction ExpressionAction::ordinaryJoin(std::shared_ptr<AnalyzedJoin> table_join)
{
ExpressionAction a;
a.type = JOIN;
a.join_params = join_params;
a.join = hash_join;
a.table_join = table_join;
return a;
}
@ -269,7 +269,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
case JOIN:
{
join_params->addJoinedColumnsAndCorrectNullability(sample_block);
table_join->addJoinedColumnsAndCorrectNullability(sample_block);
break;
}
@ -475,7 +475,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
case JOIN:
{
join->joinBlock(block, *join_params);
table_join->joinBlock(block);
break;
}
@ -543,7 +543,7 @@ void ExpressionAction::executeOnTotals(Block & block) const
if (type != JOIN)
execute(block, false);
else
join->joinTotals(block);
table_join->joinTotals(block);
}
@ -593,10 +593,10 @@ std::string ExpressionAction::toString() const
case JOIN:
ss << "JOIN ";
for (NamesAndTypesList::const_iterator it = join_params->columnsAddedByJoin().begin();
it != join_params->columnsAddedByJoin().end(); ++it)
for (NamesAndTypesList::const_iterator it = table_join->columnsAddedByJoin().begin();
it != table_join->columnsAddedByJoin().end(); ++it)
{
if (it != join_params->columnsAddedByJoin().begin())
if (it != table_join->columnsAddedByJoin().begin())
ss << ", ";
ss << it->name;
}
@ -762,17 +762,10 @@ void ExpressionActions::execute(Block & block, bool dry_run) const
bool ExpressionActions::hasTotalsInJoin() const
{
bool has_totals_in_join = false;
for (const auto & action : actions)
{
if (action.join && action.join->hasTotals())
{
has_totals_in_join = true;
break;
}
}
return has_totals_in_join;
if (action.table_join && action.table_join->hasTotals())
return true;
return false;
}
void ExpressionActions::executeOnTotals(Block & block) const
@ -1164,13 +1157,11 @@ void ExpressionActions::optimizeArrayJoin()
}
BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRightJoin(const Block & source_header, UInt64 max_block_size) const
std::shared_ptr<const AnalyzedJoin> ExpressionActions::getTableJoin() const
{
for (const auto & action : actions)
if (action.join && isRightOrFull(action.join->getKind()))
return action.join->createStreamWithNonJoinedRows(
source_header, *action.join_params, max_block_size);
if (action.table_join)
return action.table_join;
return {};
}
@ -1216,7 +1207,7 @@ UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action
hash.update(col);
break;
case JOIN:
for (const auto & col : action.join_params->columnsAddedByJoin())
for (const auto & col : action.table_join->columnsAddedByJoin())
hash.update(col.name);
break;
case PROJECT:
@ -1274,8 +1265,7 @@ bool ExpressionAction::operator==(const ExpressionAction & other) const
&& argument_names == other.argument_names
&& array_joined_columns == other.array_joined_columns
&& array_join_is_left == other.array_join_is_left
&& join == other.join
&& AnalyzedJoin::sameJoin(join_params.get(), other.join_params.get())
&& AnalyzedJoin::sameJoin(table_join.get(), other.table_join.get())
&& projection == other.projection
&& is_function_compiled == other.is_function_compiled;
}

View File

@ -4,9 +4,7 @@
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Names.h>
#include <Core/Settings.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <Interpreters/Context.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Common/SipHash.h>
#include "config_core.h"
#include <unordered_map>
@ -25,7 +23,7 @@ namespace ErrorCodes
using NameWithAlias = std::pair<std::string, std::string>;
using NamesWithAliases = std::vector<NameWithAlias>;
class Join;
class AnalyzedJoin;
class IPreparedFunction;
using PreparedFunctionPtr = std::shared_ptr<IPreparedFunction>;
@ -105,8 +103,7 @@ public:
bool unaligned_array_join = false;
/// For JOIN
std::shared_ptr<AnalyzedJoin> join_params = nullptr;
std::shared_ptr<const Join> join;
std::shared_ptr<const AnalyzedJoin> table_join;
/// For PROJECT.
NamesWithAliases projection;
@ -122,7 +119,7 @@ public:
static ExpressionAction project(const Names & projected_columns_);
static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_);
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context);
static ExpressionAction ordinaryJoin(std::shared_ptr<AnalyzedJoin> join_params, std::shared_ptr<const Join> hash_join);
static ExpressionAction ordinaryJoin(std::shared_ptr<AnalyzedJoin> join);
/// Which columns necessary to perform this action.
Names getNeededColumns() const;
@ -238,7 +235,7 @@ public:
static std::string getSmallestColumn(const NamesAndTypesList & columns);
BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin(const Block & source_header, UInt64 max_block_size) const;
std::shared_ptr<const AnalyzedJoin> getTableJoin() const;
const Settings & getSettings() const { return settings; }

View File

@ -406,10 +406,9 @@ bool SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & cha
return true;
}
/// It's possible to set nullptr as join for only_types mode
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, JoinPtr join) const
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions) const
{
actions->add(ExpressionAction::ordinaryJoin(syntax->analyzed_join, join));
actions->add(ExpressionAction::ordinaryJoin(syntax->analyzed_join));
}
bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
@ -419,13 +418,13 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b
return false;
SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join);
syntax->analyzed_join->setHashJoin(subquery_for_set.join);
initChain(chain, sourceColumns());
ExpressionActionsChain::Step & step = chain.steps.back();
getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions);
addJoinAction(step.actions, subquery_for_set.join);
addJoinAction(step.actions);
return true;
}

View File

@ -130,7 +130,7 @@ protected:
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
void addJoinAction(ExpressionActionsPtr & actions, JoinPtr join = {}) const;
void addJoinAction(ExpressionActionsPtr & actions) const;
void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false);

View File

@ -82,6 +82,7 @@
#include <Processors/Transforms/RollupTransform.h>
#include <Processors/Transforms/CubeTransform.h>
#include <Processors/LimitTransform.h>
#include <Processors/Transforms/FinishSortingTransform.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataStreams/materializeBlock.h>
@ -1037,12 +1038,10 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
stream = std::make_shared<ExpressionBlockInputStream>(stream, expressions.before_join);
}
const auto & join = query.join()->table_join->as<ASTTableJoin &>();
if (isRightOrFull(join.kind))
if (auto join = expressions.before_join->getTableJoin())
{
if (auto stream = join->createStreamWithNonJoinedDataIfFullOrRightJoin(header_before_join, settings.max_block_size))
{
auto stream = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(
header_before_join, settings.max_block_size);
if constexpr (pipeline_with_processors)
{
auto source = std::make_shared<SourceFromInputStream>(std::move(stream));
@ -1050,7 +1049,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
}
else
pipeline.stream_with_non_joined_data = std::move(stream);
}
}
}
@ -2077,10 +2076,8 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so
}
}
void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoPtr /* sorting_info */)
void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info)
{
/// TODO: Implement optimization using sorting_info
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query);
UInt64 limit = getLimitForSorting(query, context);
@ -2092,6 +2089,50 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP
// limits.mode = IBlockInputStream::LIMITS_TOTAL;
// limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode);
if (sorting_info)
{
/* Case of sorting with optimization using sorting key.
* We have several threads, each of them reads batch of parts in direct
* or reverse order of sorting key using one input stream per part
* and then merge them into one sorted stream.
* At this stage we merge per-thread streams into one.
*/
bool need_finish_sorting = (sorting_info->prefix_order_descr.size() < order_descr.size());
if (need_finish_sorting)
{
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
{
bool do_count_rows = stream_type == QueryPipeline::StreamType::Main;
return std::make_shared<PartialSortingTransform>(header, order_descr, limit, do_count_rows);
});
}
if (pipeline.getNumStreams() > 1)
{
UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit);
auto transform = std::make_shared<MergingSortedTransform>(
pipeline.getHeader(),
pipeline.getNumStreams(),
sorting_info->prefix_order_descr,
settings.max_block_size, limit_for_merging);
pipeline.addPipe({ std::move(transform) });
}
if (need_finish_sorting)
{
pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr
{
return std::make_shared<FinishSortingTransform>(
header, sorting_info->prefix_order_descr,
order_descr, settings.max_block_size, limit);
});
}
return;
}
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
{

View File

@ -26,7 +26,7 @@
namespace DB
{
struct AnalyzedJoin;
class AnalyzedJoin;
namespace JoinStuff
{

View File

@ -6,6 +6,7 @@
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/Context.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTExpressionList.h>
@ -295,6 +296,23 @@ struct RewriteTablesVisitorData
}
};
/// Attach alias to the first visited subquery
struct SetSubqueryAliasVisitorData
{
using TypeToVisit = ASTSubquery;
const String & alias;
bool done = false;
void visit(ASTSubquery &, ASTPtr & ast)
{
if (done)
return;
ast->setAlias(alias);
done = true;
}
};
bool needRewrite(ASTSelectQuery & select, std::vector<const ASTTableExpression *> & table_expressions)
{
if (!select.tables())
@ -354,6 +372,8 @@ bool needRewrite(ASTSelectQuery & select, std::vector<const ASTTableExpression *
using RewriteMatcher = OneTypeMatcher<RewriteTablesVisitorData>;
using RewriteVisitor = InDepthNodeVisitor<RewriteMatcher, true>;
using SetSubqueryAliasMatcher = OneTypeMatcher<SetSubqueryAliasVisitorData>;
using SetSubqueryAliasVisitor = InDepthNodeVisitor<SetSubqueryAliasMatcher, true>;
using ExtractAsterisksVisitor = ExtractAsterisksMatcher::Visitor;
using ColumnAliasesVisitor = InDepthNodeVisitor<ColumnAliasesMatcher, true>;
using AppendSemanticMatcher = OneTypeMatcher<AppendSemanticVisitorData>;
@ -419,6 +439,14 @@ void JoinToSubqueryTransformMatcher::visit(ASTSelectQuery & select, ASTPtr & ast
if (!left_table)
throw Exception("Cannot replace tables with subselect", ErrorCodes::LOGICAL_ERROR);
/// attach an alias to subquery.
/// TODO: remove setting check after testing period
if (data.context.getSettingsRef().joined_subquery_requires_alias)
{
SetSubqueryAliasVisitor::Data alias_data{String("--.join") + std::to_string(i)};
SetSubqueryAliasVisitor(alias_data).visit(left_table);
}
/// attach data to generated asterisk
AppendSemanticVisitor::Data semantic_data{rev_aliases, false};
AppendSemanticVisitor(semantic_data).visit(left_table);

View File

@ -14,6 +14,7 @@
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/formatAST.h>
#include <IO/WriteHelpers.h>
#include "MutationsInterpreter.h"
namespace DB
@ -37,39 +38,12 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const
return true;
}
/// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query.
/// The result can differ from tne number of affected rows (e.g. if there is an UPDATE command that
/// changes how many rows satisfy the predicates of the subsequent commands).
/// But we can be sure that if count = 0, then no rows will be touched.
auto select = std::make_shared<ASTSelectQuery>();
select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
auto count_func = std::make_shared<ASTFunction>();
count_func->name = "count";
count_func->arguments = std::make_shared<ASTExpressionList>();
select->select()->children.push_back(count_func);
if (commands.size() == 1)
select->setExpression(ASTSelectQuery::Expression::WHERE, commands[0].predicate->clone());
else
{
auto coalesced_predicates = std::make_shared<ASTFunction>();
coalesced_predicates->name = "or";
coalesced_predicates->arguments = std::make_shared<ASTExpressionList>();
coalesced_predicates->children.push_back(coalesced_predicates->arguments);
for (const MutationCommand & command : commands)
coalesced_predicates->arguments->children.push_back(command.predicate->clone());
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(coalesced_predicates));
}
auto context_copy = context;
context_copy.getSettingsRef().merge_tree_uniform_read_distribution = 0;
context_copy.getSettingsRef().max_threads = 1;
BlockInputStreamPtr in = InterpreterSelectQuery(select, context_copy, storage, SelectQueryOptions().ignoreLimits()).execute().in;
const ASTPtr & select_query = prepareQueryAffectedAST();
BlockInputStreamPtr in = InterpreterSelectQuery(select_query, context_copy, storage, SelectQueryOptions().ignoreLimits()).execute().in;
Block block = in->read();
if (!block.rows())
@ -157,7 +131,7 @@ static void validateUpdateColumns(
}
void MutationsInterpreter::prepare(bool dry_run)
ASTPtr MutationsInterpreter::prepare(bool dry_run)
{
if (is_prepared)
throw Exception("MutationsInterpreter is already prepared. It is a bug.", ErrorCodes::LOGICAL_ERROR);
@ -304,7 +278,11 @@ void MutationsInterpreter::prepare(bool dry_run)
stages_copy.back().output_columns = stage.output_columns;
stages_copy.back().filters = stage.filters;
}
auto first_stage_header = prepareInterpreterSelect(stages_copy, /* dry_run = */ true)->getSampleBlock();
const ASTPtr select_query = prepareInterpreterSelectQuery(stages_copy, /* dry_run = */ true);
InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()};
auto first_stage_header = interpreter.getSampleBlock();
auto in = std::make_shared<NullBlockInputStream>(first_stage_header);
updated_header = std::make_unique<Block>(addStreamsForLaterStages(stages_copy, in)->getHeader());
}
@ -315,12 +293,12 @@ void MutationsInterpreter::prepare(bool dry_run)
column, std::make_shared<ASTIdentifier>(column));
}
interpreter_select = prepareInterpreterSelect(stages, dry_run);
is_prepared = true;
return prepareInterpreterSelectQuery(stages, dry_run);
}
std::unique_ptr<InterpreterSelectQuery> MutationsInterpreter::prepareInterpreterSelect(std::vector<Stage> & prepared_stages, bool dry_run)
ASTPtr MutationsInterpreter::prepareInterpreterSelectQuery(std::vector<Stage> &prepared_stages, bool dry_run)
{
NamesAndTypesList all_columns = storage->getColumns().getAllPhysical();
@ -426,7 +404,7 @@ std::unique_ptr<InterpreterSelectQuery> MutationsInterpreter::prepareInterpreter
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(where_expression));
}
return std::make_unique<InterpreterSelectQuery>(select, context, storage, SelectQueryOptions().analyze(dry_run).ignoreLimits());
return select;
}
BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::vector<Stage> & prepared_stages, BlockInputStreamPtr in) const
@ -462,17 +440,19 @@ BlockInputStreamPtr MutationsInterpreter::addStreamsForLaterStages(const std::ve
void MutationsInterpreter::validate(TableStructureReadLockHolder &)
{
prepare(/* dry_run = */ true);
const auto & select_query = prepare(/* dry_run = */ true);
InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ true).ignoreLimits()};
/// Do not use getSampleBlock in order to check the whole pipeline.
Block first_stage_header = interpreter_select->execute().in->getHeader();
Block first_stage_header = interpreter.execute().in->getHeader();
BlockInputStreamPtr in = std::make_shared<NullBlockInputStream>(first_stage_header);
addStreamsForLaterStages(stages, in)->getHeader();
}
BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder &)
{
prepare(/* dry_run = */ false);
BlockInputStreamPtr in = interpreter_select->execute().in;
const auto & select_query = prepare(/* dry_run = */ false);
InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()};
BlockInputStreamPtr in = interpreter.execute().in;
auto result_stream = addStreamsForLaterStages(stages, in);
if (!updated_header)
updated_header = std::make_unique<Block>(result_stream->getHeader());
@ -484,4 +464,46 @@ const Block & MutationsInterpreter::getUpdatedHeader() const
return *updated_header;
}
ASTPtr MutationsInterpreter::prepareQueryAffectedAST() const
{
/// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query.
/// The result can differ from tne number of affected rows (e.g. if there is an UPDATE command that
/// changes how many rows satisfy the predicates of the subsequent commands).
/// But we can be sure that if count = 0, then no rows will be touched.
auto select = std::make_shared<ASTSelectQuery>();
select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared<ASTExpressionList>());
auto count_func = std::make_shared<ASTFunction>();
count_func->name = "count";
count_func->arguments = std::make_shared<ASTExpressionList>();
select->select()->children.push_back(count_func);
if (commands.size() == 1)
select->setExpression(ASTSelectQuery::Expression::WHERE, commands[0].predicate->clone());
else
{
auto coalesced_predicates = std::make_shared<ASTFunction>();
coalesced_predicates->name = "or";
coalesced_predicates->arguments = std::make_shared<ASTExpressionList>();
coalesced_predicates->children.push_back(coalesced_predicates->arguments);
for (const MutationCommand & command : commands)
coalesced_predicates->arguments->children.push_back(command.predicate->clone());
select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(coalesced_predicates));
}
return select;
}
size_t MutationsInterpreter::evaluateCommandsSize()
{
for (const MutationCommand & command : commands)
if (unlikely(!command.predicate)) /// The command touches all rows.
return prepare(/* dry_run = */ true)->size();
return std::max(prepareQueryAffectedAST()->size(), prepare(/* dry_run = */ true)->size());
}
}

View File

@ -27,6 +27,8 @@ public:
void validate(TableStructureReadLockHolder & table_lock_holder);
size_t evaluateCommandsSize();
/// Return false if the data isn't going to be changed by mutations.
bool isStorageTouchedByMutations() const;
@ -37,11 +39,12 @@ public:
const Block & getUpdatedHeader() const;
private:
void prepare(bool dry_run);
ASTPtr prepare(bool dry_run);
struct Stage;
std::unique_ptr<InterpreterSelectQuery> prepareInterpreterSelect(std::vector<Stage> & prepared_stages, bool dry_run);
ASTPtr prepareQueryAffectedAST() const;
ASTPtr prepareInterpreterSelectQuery(std::vector<Stage> &prepared_stages, bool dry_run);
BlockInputStreamPtr addStreamsForLaterStages(const std::vector<Stage> & prepared_stages, BlockInputStreamPtr in) const;
StoragePtr storage;
@ -82,7 +85,6 @@ private:
Names filter_column_names;
};
std::unique_ptr<InterpreterSelectQuery> interpreter_select;
std::unique_ptr<Block> updated_header;
std::vector<Stage> stages;
bool is_prepared = false; /// Has the sequence of stages been prepared.

View File

@ -50,6 +50,7 @@ namespace ErrorCodes
extern const int NOT_IMPLEMENTED;
extern const int UNKNOWN_IDENTIFIER;
extern const int EXPECTED_ALL_OR_ANY;
extern const int ALIAS_REQUIRED;
}
NameSet removeDuplicateColumns(NamesAndTypesList & columns)
@ -99,14 +100,22 @@ void translateQualifiedNames(ASTPtr & query, const ASTSelectQuery & select_query
const Names & source_columns_list, const NameSet & source_columns_set,
const NameSet & columns_from_joined_table)
{
auto & settings = context.getSettingsRef();
std::vector<TableWithColumnNames> tables_with_columns = getDatabaseAndTablesWithColumnNames(select_query, context);
if (settings.joined_subquery_requires_alias && tables_with_columns.size() > 1)
{
for (auto & pr : tables_with_columns)
if (pr.first.table.empty() && pr.first.alias.empty())
throw Exception("Not unique subquery in FROM requires an alias (or joined_subquery_requires_alias=0 to disable restriction).",
ErrorCodes::ALIAS_REQUIRED);
}
if (tables_with_columns.empty())
{
Names all_columns_name = source_columns_list;
/// TODO: asterisk_left_columns_only probably does not work in some cases
if (!context.getSettingsRef().asterisk_left_columns_only)
if (!settings.asterisk_left_columns_only)
{
for (auto & column : columns_from_joined_table)
all_columns_name.emplace_back(column);

View File

@ -28,12 +28,19 @@
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/executeQuery.h>
#include <Common/ProfileEvents.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Formats/IOutputFormat.h>
namespace ProfileEvents
{
extern const Event QueryMaskingRulesMatch;
}
namespace DB
{
@ -54,7 +61,6 @@ static void checkASTSizeLimits(const IAST & ast, const Settings & settings)
ast.checkSize(settings.max_ast_elements);
}
/// NOTE This is wrong in case of single-line comments and in case of multiline string literals.
static String joinLines(const String & query)
{
@ -64,6 +70,27 @@ static String joinLines(const String & query)
}
static String prepareQueryForLogging(const String & query, Context & context)
{
String res = query;
// wiping sensitive data before cropping query by log_queries_cut_to_length,
// otherwise something like credit card without last digit can go to log
if (auto masker = context.getSensitiveDataMasker())
{
auto matches = masker->wipeSensitiveData(res);
if (matches > 0)
{
ProfileEvents::increment(ProfileEvents::QueryMaskingRulesMatch, matches);
}
}
res = res.substr(0, context.getSettingsRef().log_queries_cut_to_length);
return res;
}
/// Log query into text log (not into system table).
static void logQuery(const String & query, const Context & context, bool internal)
{
@ -111,7 +138,7 @@ static void logException(Context & context, QueryLogElement & elem)
}
static void onExceptionBeforeStart(const String & query, Context & context, time_t current_time)
static void onExceptionBeforeStart(const String & query_for_logging, Context & context, time_t current_time)
{
/// Exception before the query execution.
context.getQuota().addError();
@ -126,7 +153,7 @@ static void onExceptionBeforeStart(const String & query, Context & context, time
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query = query.substr(0, settings.log_queries_cut_to_length);
elem.query = query_for_logging;
elem.exception = getCurrentExceptionMessage(false);
elem.client_info = context.getClientInfo();
@ -192,10 +219,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
{
/// Anyway log the query.
String query = String(begin, begin + std::min(end - begin, static_cast<ptrdiff_t>(max_query_size)));
logQuery(query.substr(0, settings.log_queries_cut_to_length), context, internal);
auto query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal);
if (!internal)
onExceptionBeforeStart(query, context, current_time);
onExceptionBeforeStart(query_for_logging, context, current_time);
throw;
}
@ -205,6 +234,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
BlockIO res;
QueryPipeline & pipeline = res.pipeline;
String query_for_logging = "";
try
{
/// Replace ASTQueryParameter with ASTLiteral for prepared statements.
@ -217,7 +248,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
query = serializeAST(*ast);
}
logQuery(query.substr(0, settings.log_queries_cut_to_length), context, internal);
query_for_logging = prepareQueryForLogging(query, context);
logQuery(query_for_logging, context, internal);
/// Check the limits.
checkASTSizeLimits(*ast, settings);
@ -231,7 +264,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
ProcessList::EntryPtr process_list_entry;
if (!internal && !ast->as<ASTShowProcesslistQuery>())
{
process_list_entry = context.getProcessList().insert(query, ast.get(), context);
/// processlist also has query masked now, to avoid secrets leaks though SHOW PROCESSLIST by other users.
process_list_entry = context.getProcessList().insert(query_for_logging, ast.get(), context);
context.setProcessListElement(&process_list_entry->get());
}
@ -323,7 +357,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
elem.event_time = current_time;
elem.query_start_time = current_time;
elem.query = query.substr(0, settings.log_queries_cut_to_length);
elem.query = query_for_logging;
elem.client_info = context.getClientInfo();
@ -469,7 +503,12 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
catch (...)
{
if (!internal)
onExceptionBeforeStart(query, context, current_time);
{
if (query_for_logging.empty())
query_for_logging = prepareQueryForLogging(query, context);
onExceptionBeforeStart(query_for_logging, context, current_time);
}
throw;
}

View File

@ -46,6 +46,14 @@ String backQuote(const String & x)
return res;
}
size_t IAST::size() const
{
size_t res = 1;
for (const auto & child : children)
res += child->size();
return res;
}
size_t IAST::checkSize(size_t max_size) const
{

View File

@ -93,6 +93,10 @@ public:
return checkDepthImpl(max_depth, 0);
}
/** Get total number of tree elements
*/
size_t size() const;
/** Same for the total number of tree elements.
*/
size_t checkSize(size_t max_size) const;

View File

@ -0,0 +1,141 @@
#include <Processors/Transforms/FinishSortingTransform.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
static bool isPrefix(const SortDescription & pref_descr, const SortDescription & descr)
{
if (pref_descr.size() > descr.size())
return false;
for (size_t i = 0; i < pref_descr.size(); ++i)
if (pref_descr[i] != descr[i])
return false;
return true;
}
FinishSortingTransform::FinishSortingTransform(
const Block & header, const SortDescription & description_sorted_,
const SortDescription & description_to_sort_,
size_t max_merged_block_size_, UInt64 limit_)
: SortingTransform(header, description_to_sort_, max_merged_block_size_, limit_)
, description_sorted(description_sorted_)
{
const auto & sample = inputs.front().getHeader();
/// Replace column names to column position in description_sorted.
for (auto & column_description : description_sorted)
{
if (!column_description.column_name.empty())
{
column_description.column_number = sample.getPositionByName(column_description.column_name);
column_description.column_name.clear();
}
}
if (!isPrefix(description_sorted, description))
throw Exception("Can`t finish sorting. SortDescription of already sorted stream is not prefix of "
"SortDescription needed to sort", ErrorCodes::LOGICAL_ERROR);
}
static bool less(const Columns & lhs, const Columns & rhs, size_t i, size_t j, const SortDescription & descr)
{
for (const auto & elem : descr)
{
size_t ind = elem.column_number;
int res = elem.direction * lhs[ind]->compareAt(i, j, *rhs[ind], elem.nulls_direction);
if (res < 0)
return true;
else if (res > 0)
return false;
}
return false;
}
void FinishSortingTransform::consume(Chunk chunk)
{
generated_prefix = false;
// If there were only const columns in sort description, then there is no need to sort.
// Return the chunks as is.
if (description.empty())
{
generated_chunk = std::move(chunk);
return;
}
removeConstColumns(chunk);
/// Find the position of last already read key in current chunk.
if (!chunks.empty())
{
size_t size = chunk.getNumRows();
const auto & last_chunk = chunks.back();
ssize_t low = -1;
ssize_t high = size;
while (high - low > 1)
{
ssize_t mid = (low + high) / 2;
if (!less(last_chunk.getColumns(), chunk.getColumns(), last_chunk.getNumRows() - 1, mid, description_sorted))
low = mid;
else
high = mid;
}
size_t tail_pos = high;
/// We need to save tail of chunk, because next chunk may starts with the same key as in tail
/// and we should sort these rows in one portion.
if (tail_pos != size)
{
auto source_columns = chunk.detachColumns();
Columns tail_columns;
for (size_t i = 0; i < source_columns.size(); ++i)
{
tail_columns.push_back(source_columns[i]->cut(tail_pos, size - tail_pos));
source_columns[i] = source_columns[i]->cut(0, tail_pos);
}
chunks.emplace_back(std::move(source_columns), tail_pos);
tail_chunk.setColumns(std::move(tail_columns), size - tail_pos);
stage = Stage::Generate;
return;
}
}
/// If we reach here, that means that current cunk is first in portion
/// or it all consists of rows with the same key as tail of a previous chunk.
chunks.push_back(std::move(chunk));
}
void FinishSortingTransform::generate()
{
if (!merge_sorter)
{
merge_sorter = std::make_unique<MergeSorter>(std::move(chunks), description, max_merged_block_size, limit);
generated_prefix = true;
}
generated_chunk = merge_sorter->read();
if (!generated_chunk)
{
merge_sorter.reset();
if (tail_chunk)
chunks.push_back(std::move(tail_chunk));
stage = Stage::Consume;
}
else
enrichChunkWithConstants(generated_chunk);
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Processors/Transforms/SortingTransform.h>
namespace DB
{
/** Takes stream already sorted by `x` and finishes sorting it by (`x`, `y`).
* During sorting only chunks with rows that equal by `x` saved in RAM.
* */
class FinishSortingTransform : public SortingTransform
{
public:
/// limit - if not 0, allowed to return just first 'limit' rows in sorted order.
FinishSortingTransform(const Block & header, const SortDescription & description_sorted_,
const SortDescription & description_to_sort_,
size_t max_merged_block_size_, UInt64 limit_);
String getName() const override { return "FinishSortingTransform"; }
protected:
void consume(Chunk chunk) override;
void generate() override;
private:
SortDescription description_sorted;
Chunk tail_chunk;
};
}

View File

@ -1,7 +1,6 @@
#include <Processors/Transforms/MergeSortingTransform.h>
#include <Core/SortDescription.h>
#include <Core/SortCursor.h>
#include <Processors/IAccumulatingTransform.h>
#include <Processors/Transforms/MergingSortedTransform.h>
#include <Common/formatReadable.h>
#include <Common/ProfileEvents.h>
@ -13,10 +12,6 @@
#include <DataStreams/NativeBlockInputStream.h>
#include <DataStreams/NativeBlockOutputStream.h>
#include <queue>
#include <Processors/ISource.h>
#include <Processors/Transforms/MergingSortedTransform.h>
namespace ProfileEvents
{
@ -91,351 +86,17 @@ private:
BlockInputStreamPtr block_in;
};
/** Part of implementation. Merging array of ready (already read from somewhere) chunks.
* Returns result of merge as stream of chunks, not more than 'max_merged_block_size' rows in each.
*/
class MergeSorter
{
public:
MergeSorter(Chunks chunks_, SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_);
Chunk read();
private:
Chunks chunks;
SortDescription description;
size_t max_merged_block_size;
UInt64 limit;
size_t total_merged_rows = 0;
using CursorImpls = std::vector<SortCursorImpl>;
CursorImpls cursors;
bool has_collation = false;
std::priority_queue<SortCursor> queue_without_collation;
std::priority_queue<SortCursorWithCollation> queue_with_collation;
/** Two different cursors are supported - with and without Collation.
* Templates are used (instead of virtual functions in SortCursor) for zero-overhead.
*/
template <typename TSortCursor>
Chunk mergeImpl(std::priority_queue<TSortCursor> & queue);
};
class MergeSorterSource : public ISource
{
public:
MergeSorterSource(Block header, Chunks chunks, SortDescription & description, size_t max_merged_block_size, UInt64 limit)
: ISource(std::move(header)), merge_sorter(std::move(chunks), description, max_merged_block_size, limit) {}
String getName() const override { return "MergeSorterSource"; }
protected:
Chunk generate() override { return merge_sorter.read(); }
private:
MergeSorter merge_sorter;
};
MergeSorter::MergeSorter(Chunks chunks_, SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_)
: chunks(std::move(chunks_)), description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_)
{
Chunks nonempty_chunks;
for (auto & chunk : chunks)
{
if (chunk.getNumRows() == 0)
continue;
cursors.emplace_back(chunk.getColumns(), description);
has_collation |= cursors.back().has_collation;
nonempty_chunks.emplace_back(std::move(chunk));
}
chunks.swap(nonempty_chunks);
if (!has_collation)
{
for (auto & cursor : cursors)
queue_without_collation.push(SortCursor(&cursor));
}
else
{
for (auto & cursor : cursors)
queue_with_collation.push(SortCursorWithCollation(&cursor));
}
}
Chunk MergeSorter::read()
{
if (chunks.empty())
return Chunk();
if (chunks.size() == 1)
{
auto res = std::move(chunks[0]);
chunks.clear();
return res;
}
return !has_collation
? mergeImpl<SortCursor>(queue_without_collation)
: mergeImpl<SortCursorWithCollation>(queue_with_collation);
}
template <typename TSortCursor>
Chunk MergeSorter::mergeImpl(std::priority_queue<TSortCursor> & queue)
{
size_t num_columns = chunks[0].getNumColumns();
MutableColumns merged_columns = chunks[0].cloneEmptyColumns();
/// TODO: reserve (in each column)
/// Take rows from queue in right order and push to 'merged'.
size_t merged_rows = 0;
while (!queue.empty())
{
TSortCursor current = queue.top();
queue.pop();
for (size_t i = 0; i < num_columns; ++i)
merged_columns[i]->insertFrom(*current->all_columns[i], current->pos);
++total_merged_rows;
++merged_rows;
if (!current->isLast())
{
current->next();
queue.push(current);
}
if (limit && total_merged_rows == limit)
{
chunks.clear();
return Chunk(std::move(merged_columns), merged_rows);
}
if (merged_rows == max_merged_block_size)
return Chunk(std::move(merged_columns), merged_rows);
}
chunks.clear();
if (merged_rows == 0)
return {};
return Chunk(std::move(merged_columns), merged_rows);
}
MergeSortingTransform::MergeSortingTransform(
const Block & header,
SortDescription & description_,
const SortDescription & description_,
size_t max_merged_block_size_, UInt64 limit_,
size_t max_bytes_before_remerge_,
size_t max_bytes_before_external_sort_, const std::string & tmp_path_,
size_t min_free_disk_space_)
: IProcessor({header}, {header})
, description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_)
: SortingTransform(header, description_, max_merged_block_size_, limit_)
, max_bytes_before_remerge(max_bytes_before_remerge_)
, max_bytes_before_external_sort(max_bytes_before_external_sort_), tmp_path(tmp_path_)
, min_free_disk_space(min_free_disk_space_)
{
auto & sample = inputs.front().getHeader();
/// Replace column names to column position in sort_description.
for (auto & column_description : description)
{
if (!column_description.column_name.empty())
{
column_description.column_number = sample.getPositionByName(column_description.column_name);
column_description.column_name.clear();
}
}
/// Remove constants from header and map old indexes to new.
size_t num_columns = sample.columns();
ColumnNumbers map(num_columns, num_columns);
const_columns_to_remove.assign(num_columns, true);
for (size_t pos = 0; pos < num_columns; ++pos)
{
const auto & column = sample.getByPosition(pos);
if (!(column.column && isColumnConst(*column.column)))
{
map[pos] = header_without_constants.columns();
header_without_constants.insert(column);
const_columns_to_remove[pos] = false;
}
}
/// Remove constants from column_description and remap positions.
SortDescription description_without_constants;
description_without_constants.reserve(description.size());
for (const auto & column_description : description)
{
auto old_pos = column_description.column_number;
auto new_pos = map[old_pos];
if (new_pos < num_columns)
{
description_without_constants.push_back(column_description);
description_without_constants.back().column_number = new_pos;
}
}
description.swap(description_without_constants);
}
MergeSortingTransform::~MergeSortingTransform() = default;
IProcessor::Status MergeSortingTransform::prepare()
{
if (stage == Stage::Serialize)
{
if (!processors.empty())
return Status::ExpandPipeline;
auto status = prepareSerialize();
if (status != Status::Finished)
return status;
stage = Stage::Consume;
}
if (stage == Stage::Consume)
{
auto status = prepareConsume();
if (status != Status::Finished)
return status;
stage = Stage::Generate;
}
/// stage == Stage::Generate
if (!generated_prefix)
return Status::Ready;
if (!processors.empty())
return Status::ExpandPipeline;
return prepareGenerate();
}
IProcessor::Status MergeSortingTransform::prepareConsume()
{
auto & input = inputs.front();
auto & output = outputs.front();
/// Check can output.
if (output.isFinished())
{
input.close();
return Status::Finished;
}
if (!output.canPush())
{
input.setNotNeeded();
return Status::PortFull;
}
if (generated_chunk)
output.push(std::move(generated_chunk));
/// Check can input.
if (!current_chunk)
{
if (input.isFinished())
return Status::Finished;
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
current_chunk = input.pull();
}
/// Now consume.
return Status::Ready;
}
IProcessor::Status MergeSortingTransform::prepareSerialize()
{
auto & output = outputs.back();
if (output.isFinished())
return Status::Finished;
if (!output.canPush())
return Status::PortFull;
if (current_chunk)
output.push(std::move(current_chunk));
if (merge_sorter)
return Status::Ready;
output.finish();
return Status::Finished;
}
IProcessor::Status MergeSortingTransform::prepareGenerate()
{
auto & output = outputs.front();
if (output.isFinished())
return Status::Finished;
if (!output.canPush())
return Status::PortFull;
if (merge_sorter)
{
if (!generated_chunk)
return Status::Ready;
output.push(std::move(generated_chunk));
return Status::PortFull;
}
else
{
auto & input = inputs.back();
if (input.isFinished())
{
output.finish();
return Status::Finished;
}
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
auto chunk = input.pull();
enrichChunkWithConstants(chunk);
output.push(std::move(chunk));
return Status::PortFull;
}
}
void MergeSortingTransform::work()
{
if (stage == Stage::Consume)
consume(std::move(current_chunk));
if (stage == Stage::Serialize)
serialize();
if (stage == Stage::Generate)
generate();
}
, min_free_disk_space(min_free_disk_space_) {}
Processors MergeSortingTransform::expandPipeline()
{
@ -612,57 +273,4 @@ void MergeSortingTransform::remerge()
sum_bytes_in_blocks = new_sum_bytes_in_blocks;
}
void MergeSortingTransform::removeConstColumns(Chunk & chunk)
{
size_t num_columns = chunk.getNumColumns();
size_t num_rows = chunk.getNumRows();
if (num_columns != const_columns_to_remove.size())
throw Exception("Block has different number of columns with header: " + toString(num_columns)
+ " vs " + toString(const_columns_to_remove.size()), ErrorCodes::LOGICAL_ERROR);
auto columns = chunk.detachColumns();
Columns column_without_constants;
column_without_constants.reserve(header_without_constants.columns());
for (size_t position = 0; position < num_columns; ++position)
{
if (!const_columns_to_remove[position])
column_without_constants.push_back(std::move(columns[position]));
}
chunk.setColumns(std::move(column_without_constants), num_rows);
}
void MergeSortingTransform::enrichChunkWithConstants(Chunk & chunk)
{
size_t num_rows = chunk.getNumRows();
size_t num_result_columns = const_columns_to_remove.size();
auto columns = chunk.detachColumns();
Columns column_with_constants;
column_with_constants.reserve(num_result_columns);
auto & header = inputs.front().getHeader();
size_t next_non_const_column = 0;
for (size_t i = 0; i < num_result_columns; ++i)
{
if (const_columns_to_remove[i])
column_with_constants.emplace_back(header.getByPosition(i).column->cloneResized(num_rows));
else
{
if (next_non_const_column >= columns.size())
throw Exception("Can't enrich chunk with constants because run out of non-constant columns.",
ErrorCodes::LOGICAL_ERROR);
column_with_constants.emplace_back(std::move(columns[next_non_const_column]));
++next_non_const_column;
}
}
chunk.setColumns(std::move(column_with_constants), num_rows);
}
}

View File

@ -1,5 +1,5 @@
#pragma once
#include <Processors/IAccumulatingTransform.h>
#include <Processors/Transforms/SortingTransform.h>
#include <Core/SortDescription.h>
#include <Poco/TemporaryFile.h>
#include <IO/ReadBufferFromFile.h>
@ -20,85 +20,44 @@ namespace ErrorCodes
}
class MergeSorter;
class MergeSortingTransform : public IProcessor
class MergeSortingTransform : public SortingTransform
{
public:
/// limit - if not 0, allowed to return just first 'limit' rows in sorted order.
MergeSortingTransform(const Block & header,
SortDescription & description_,
const SortDescription & description_,
size_t max_merged_block_size_, UInt64 limit_,
size_t max_bytes_before_remerge_,
size_t max_bytes_before_external_sort_, const std::string & tmp_path_,
size_t min_free_disk_space_);
~MergeSortingTransform() override;
String getName() const override { return "MergeSortingTransform"; }
protected:
void consume(Chunk chunk) override;
void serialize() override;
void generate() override;
Status prepare() override;
void work() override;
Processors expandPipeline() override;
private:
SortDescription description;
size_t max_merged_block_size;
UInt64 limit;
size_t max_bytes_before_remerge;
size_t max_bytes_before_external_sort;
const std::string tmp_path;
size_t min_free_disk_space;
Logger * log = &Logger::get("MergeSortingBlockInputStream");
Chunks chunks;
size_t sum_rows_in_blocks = 0;
size_t sum_bytes_in_blocks = 0;
Logger * log = &Logger::get("MergeSortingTransform");
/// If remerge doesn't save memory at least several times, mark it as useless and don't do it anymore.
bool remerge_is_useful = true;
/// Before operation, will remove constant columns from blocks. And after, place constant columns back.
/// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files)
/// Save original block structure here.
Block header_without_constants;
/// Columns which were constant in header and we need to remove from chunks.
std::vector<bool> const_columns_to_remove;
/// Everything below is for external sorting.
std::vector<std::unique_ptr<Poco::TemporaryFile>> temporary_files;
/// Merge all accumulated blocks to keep no more than limit rows.
void remerge();
void removeConstColumns(Chunk & chunk);
void enrichChunkWithConstants(Chunk & chunk);
enum class Stage
{
Consume = 0,
Generate,
Serialize,
};
Stage stage = Stage::Consume;
bool generated_prefix = false;
Chunk current_chunk;
Chunk generated_chunk;
std::unique_ptr<MergeSorter> merge_sorter;
ProcessorPtr external_merging_sorted;
Processors processors;
Status prepareConsume();
Status prepareSerialize();
Status prepareGenerate();
void consume(Chunk chunk);
void serialize();
void generate();
};
}

View File

@ -0,0 +1,384 @@
#include <Processors/Transforms/SortingTransform.h>
#include <Core/SortDescription.h>
#include <Core/SortCursor.h>
#include <Common/formatReadable.h>
#include <Common/ProfileEvents.h>
#include <IO/WriteBufferFromFile.h>
#include <Compression/CompressedWriteBuffer.h>
#include <DataStreams/NativeBlockInputStream.h>
#include <DataStreams/NativeBlockOutputStream.h>
namespace ProfileEvents
{
extern const Event ExternalSortWritePart;
extern const Event ExternalSortMerge;
}
namespace DB
{
MergeSorter::MergeSorter(Chunks chunks_, SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_)
: chunks(std::move(chunks_)), description(description_), max_merged_block_size(max_merged_block_size_), limit(limit_)
{
Chunks nonempty_chunks;
for (auto & chunk : chunks)
{
if (chunk.getNumRows() == 0)
continue;
cursors.emplace_back(chunk.getColumns(), description);
has_collation |= cursors.back().has_collation;
nonempty_chunks.emplace_back(std::move(chunk));
}
chunks.swap(nonempty_chunks);
if (!has_collation)
{
for (auto & cursor : cursors)
queue_without_collation.push(SortCursor(&cursor));
}
else
{
for (auto & cursor : cursors)
queue_with_collation.push(SortCursorWithCollation(&cursor));
}
}
Chunk MergeSorter::read()
{
if (chunks.empty())
return Chunk();
if (chunks.size() == 1)
{
auto res = std::move(chunks[0]);
chunks.clear();
return res;
}
return !has_collation
? mergeImpl<SortCursor>(queue_without_collation)
: mergeImpl<SortCursorWithCollation>(queue_with_collation);
}
template <typename TSortCursor>
Chunk MergeSorter::mergeImpl(std::priority_queue<TSortCursor> & queue)
{
size_t num_columns = chunks[0].getNumColumns();
MutableColumns merged_columns = chunks[0].cloneEmptyColumns();
/// TODO: reserve (in each column)
/// Take rows from queue in right order and push to 'merged'.
size_t merged_rows = 0;
while (!queue.empty())
{
TSortCursor current = queue.top();
queue.pop();
for (size_t i = 0; i < num_columns; ++i)
merged_columns[i]->insertFrom(*current->all_columns[i], current->pos);
++total_merged_rows;
++merged_rows;
if (!current->isLast())
{
current->next();
queue.push(current);
}
if (limit && total_merged_rows == limit)
{
chunks.clear();
return Chunk(std::move(merged_columns), merged_rows);
}
if (merged_rows == max_merged_block_size)
return Chunk(std::move(merged_columns), merged_rows);
}
chunks.clear();
if (merged_rows == 0)
return {};
return Chunk(std::move(merged_columns), merged_rows);
}
SortingTransform::SortingTransform(
const Block & header,
const SortDescription & description_,
size_t max_merged_block_size_, UInt64 limit_)
: IProcessor({header}, {header})
, description(description_)
, max_merged_block_size(max_merged_block_size_)
, limit(limit_)
{
auto & sample = inputs.front().getHeader();
/// Replace column names to column position in sort_description.
for (auto & column_description : description)
{
if (!column_description.column_name.empty())
{
column_description.column_number = sample.getPositionByName(column_description.column_name);
column_description.column_name.clear();
}
}
/// Remove constants from header and map old indexes to new.
size_t num_columns = sample.columns();
ColumnNumbers map(num_columns, num_columns);
const_columns_to_remove.assign(num_columns, true);
for (size_t pos = 0; pos < num_columns; ++pos)
{
const auto & column = sample.getByPosition(pos);
if (!(column.column && isColumnConst(*column.column)))
{
map[pos] = header_without_constants.columns();
header_without_constants.insert(column);
const_columns_to_remove[pos] = false;
}
}
/// Remove constants from column_description and remap positions.
SortDescription description_without_constants;
description_without_constants.reserve(description.size());
for (const auto & column_description : description)
{
auto old_pos = column_description.column_number;
auto new_pos = map[old_pos];
if (new_pos < num_columns)
{
description_without_constants.push_back(column_description);
description_without_constants.back().column_number = new_pos;
}
}
description.swap(description_without_constants);
}
SortingTransform::~SortingTransform() = default;
IProcessor::Status SortingTransform::prepare()
{
if (stage == Stage::Serialize)
{
if (!processors.empty())
return Status::ExpandPipeline;
auto status = prepareSerialize();
if (status != Status::Finished)
return status;
stage = Stage::Consume;
}
if (stage == Stage::Consume)
{
auto status = prepareConsume();
if (status != Status::Finished)
return status;
stage = Stage::Generate;
}
/// stage == Stage::Generate
if (!generated_prefix || !chunks.empty())
return Status::Ready;
if (!processors.empty())
return Status::ExpandPipeline;
return prepareGenerate();
}
IProcessor::Status SortingTransform::prepareConsume()
{
auto & input = inputs.front();
auto & output = outputs.front();
/// Check can output.
if (output.isFinished())
{
input.close();
return Status::Finished;
}
if (!output.canPush())
{
input.setNotNeeded();
return Status::PortFull;
}
if (generated_chunk)
output.push(std::move(generated_chunk));
/// Check can input.
if (!current_chunk)
{
if (input.isFinished())
return Status::Finished;
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
current_chunk = input.pull();
}
/// Now consume.
return Status::Ready;
}
IProcessor::Status SortingTransform::prepareSerialize()
{
auto & output = outputs.back();
if (output.isFinished())
return Status::Finished;
if (!output.canPush())
return Status::PortFull;
if (current_chunk)
output.push(std::move(current_chunk));
if (merge_sorter)
return Status::Ready;
output.finish();
return Status::Finished;
}
IProcessor::Status SortingTransform::prepareGenerate()
{
auto & output = outputs.front();
if (output.isFinished())
{
inputs.front().close();
return Status::Finished;
}
if (!output.canPush())
return Status::PortFull;
if (merge_sorter)
{
if (!generated_chunk)
return Status::Ready;
output.push(std::move(generated_chunk));
return Status::PortFull;
}
else
{
auto & input = inputs.back();
if (generated_chunk)
output.push(std::move(generated_chunk));
if (input.isFinished())
{
output.finish();
return Status::Finished;
}
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
auto chunk = input.pull();
enrichChunkWithConstants(chunk);
output.push(std::move(chunk));
return Status::PortFull;
}
}
void SortingTransform::work()
{
if (stage == Stage::Consume)
consume(std::move(current_chunk));
if (stage == Stage::Serialize)
serialize();
if (stage == Stage::Generate)
generate();
}
void SortingTransform::removeConstColumns(Chunk & chunk)
{
size_t num_columns = chunk.getNumColumns();
size_t num_rows = chunk.getNumRows();
if (num_columns != const_columns_to_remove.size())
throw Exception("Block has different number of columns with header: " + toString(num_columns)
+ " vs " + toString(const_columns_to_remove.size()), ErrorCodes::LOGICAL_ERROR);
auto columns = chunk.detachColumns();
Columns column_without_constants;
column_without_constants.reserve(header_without_constants.columns());
for (size_t position = 0; position < num_columns; ++position)
{
if (!const_columns_to_remove[position])
column_without_constants.push_back(std::move(columns[position]));
}
chunk.setColumns(std::move(column_without_constants), num_rows);
}
void SortingTransform::enrichChunkWithConstants(Chunk & chunk)
{
size_t num_rows = chunk.getNumRows();
size_t num_result_columns = const_columns_to_remove.size();
auto columns = chunk.detachColumns();
Columns column_with_constants;
column_with_constants.reserve(num_result_columns);
auto & header = inputs.front().getHeader();
size_t next_non_const_column = 0;
for (size_t i = 0; i < num_result_columns; ++i)
{
if (const_columns_to_remove[i])
column_with_constants.emplace_back(header.getByPosition(i).column->cloneResized(num_rows));
else
{
if (next_non_const_column >= columns.size())
throw Exception("Can't enrich chunk with constants because run out of non-constant columns.",
ErrorCodes::LOGICAL_ERROR);
column_with_constants.emplace_back(std::move(columns[next_non_const_column]));
++next_non_const_column;
}
}
chunk.setColumns(std::move(column_with_constants), num_rows);
}
void SortingTransform::serialize()
{
throw Exception("Method 'serialize' is not implemented for " + getName() + " processor", ErrorCodes::NOT_IMPLEMENTED);
}
}

View File

@ -0,0 +1,122 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Core/SortDescription.h>
#include <Core/SortCursor.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/ISource.h>
#include <queue>
namespace DB
{
/** Part of implementation. Merging array of ready (already read from somewhere) chunks.
* Returns result of merge as stream of chunks, not more than 'max_merged_block_size' rows in each.
*/
class MergeSorter
{
public:
MergeSorter(Chunks chunks_, SortDescription & description_, size_t max_merged_block_size_, UInt64 limit_);
Chunk read();
private:
Chunks chunks;
SortDescription description;
size_t max_merged_block_size;
UInt64 limit;
size_t total_merged_rows = 0;
using CursorImpls = std::vector<SortCursorImpl>;
CursorImpls cursors;
bool has_collation = false;
std::priority_queue<SortCursor> queue_without_collation;
std::priority_queue<SortCursorWithCollation> queue_with_collation;
/** Two different cursors are supported - with and without Collation.
* Templates are used (instead of virtual functions in SortCursor) for zero-overhead.
*/
template <typename TSortCursor>
Chunk mergeImpl(std::priority_queue<TSortCursor> & queue);
};
class MergeSorterSource : public ISource
{
public:
MergeSorterSource(Block header, Chunks chunks, SortDescription & description, size_t max_merged_block_size, UInt64 limit)
: ISource(std::move(header)), merge_sorter(std::move(chunks), description, max_merged_block_size, limit) {}
String getName() const override { return "MergeSorterSource"; }
protected:
Chunk generate() override { return merge_sorter.read(); }
private:
MergeSorter merge_sorter;
};
/** Base class for sorting.
* Currently there are two implementations: MergeSortingTransform and FinishSortingTransform.
*/
class SortingTransform : public IProcessor
{
public:
/// limit - if not 0, allowed to return just first 'limit' rows in sorted order.
SortingTransform(const Block & header,
const SortDescription & description_,
size_t max_merged_block_size_, UInt64 limit_);
~SortingTransform() override;
protected:
Status prepare() override final;
void work() override final;
virtual void consume(Chunk chunk) = 0;
virtual void generate() = 0;
virtual void serialize();
SortDescription description;
size_t max_merged_block_size;
UInt64 limit;
size_t sum_rows_in_blocks = 0;
size_t sum_bytes_in_blocks = 0;
/// Before operation, will remove constant columns from blocks. And after, place constant columns back.
/// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files)
/// Save original block structure here.
Block header_without_constants;
/// Columns which were constant in header and we need to remove from chunks.
std::vector<bool> const_columns_to_remove;
void removeConstColumns(Chunk & chunk);
void enrichChunkWithConstants(Chunk & chunk);
enum class Stage
{
Consume = 0,
Generate,
Serialize,
};
Stage stage = Stage::Consume;
bool generated_prefix = false;
Chunk current_chunk;
Chunk generated_chunk;
Chunks chunks;
std::unique_ptr<MergeSorter> merge_sorter;
Processors processors;
private:
Status prepareConsume();
Status prepareSerialize();
Status prepareGenerate();
};
}

View File

@ -28,10 +28,7 @@ KafkaBlockInputStream::~KafkaBlockInputStream()
return;
if (broken)
{
buffer->subBufferAs<ReadBufferFromKafkaConsumer>()->unsubscribe();
buffer->reset();
}
buffer->unsubscribe();
storage.pushReadBuffer(buffer);
}
@ -50,23 +47,22 @@ void KafkaBlockInputStream::readPrefixImpl()
if (!buffer)
return;
buffer->subBufferAs<ReadBufferFromKafkaConsumer>()->subscribe(storage.getTopics());
buffer->subscribe(storage.getTopics());
const auto & limits_ = getLimits();
const size_t poll_timeout = buffer->subBufferAs<ReadBufferFromKafkaConsumer>()->pollTimeout();
const size_t poll_timeout = buffer->pollTimeout();
size_t rows_portion_size = poll_timeout ? std::min<size_t>(max_block_size, limits_.max_execution_time.totalMilliseconds() / poll_timeout) : max_block_size;
rows_portion_size = std::max(rows_portion_size, 1ul);
auto non_virtual_header = storage.getSampleBlockNonMaterialized(); /// FIXME: add materialized columns support
auto read_callback = [this]
{
const auto * sub_buffer = buffer->subBufferAs<ReadBufferFromKafkaConsumer>();
virtual_columns[0]->insert(sub_buffer->currentTopic()); // "topic"
virtual_columns[1]->insert(sub_buffer->currentKey()); // "key"
virtual_columns[2]->insert(sub_buffer->currentOffset()); // "offset"
virtual_columns[3]->insert(sub_buffer->currentPartition()); // "partition"
virtual_columns[0]->insert(buffer->currentTopic()); // "topic"
virtual_columns[1]->insert(buffer->currentKey()); // "key"
virtual_columns[2]->insert(buffer->currentOffset()); // "offset"
virtual_columns[3]->insert(buffer->currentPartition()); // "partition"
auto timestamp = sub_buffer->currentTimestamp();
auto timestamp = buffer->currentTimestamp();
if (timestamp)
virtual_columns[4]->insert(std::chrono::duration_cast<std::chrono::seconds>(timestamp->get_timestamp()).count()); // "timestamp"
};
@ -106,7 +102,7 @@ void KafkaBlockInputStream::readSuffixImpl()
if (!buffer)
return;
buffer->subBufferAs<ReadBufferFromKafkaConsumer>()->commit();
buffer->commit();
broken = false;
}

View File

@ -1,5 +1,7 @@
#include <Storages/Kafka/ReadBufferFromKafkaConsumer.h>
#include <common/logger_useful.h>
namespace DB
{
@ -11,6 +13,7 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer(
size_t max_batch_size,
size_t poll_timeout_,
bool intermediate_commit_,
char delimiter_,
const std::atomic<bool> & stopped_)
: ReadBuffer(nullptr, 0)
, consumer(consumer_)
@ -18,6 +21,7 @@ ReadBufferFromKafkaConsumer::ReadBufferFromKafkaConsumer(
, batch_size(max_batch_size)
, poll_timeout(poll_timeout_)
, intermediate_commit(intermediate_commit_)
, delimiter(delimiter_)
, stopped(stopped_)
, current(messages.begin())
{
@ -68,20 +72,9 @@ void ReadBufferFromKafkaConsumer::commit()
PrintOffsets("Polled offset", consumer->get_offsets_position(consumer->get_assignment()));
if (current != messages.end())
{
/// Since we can poll more messages than we already processed,
/// commit only processed messages.
/// Since we can poll more messages than we already processed - commit only processed messages.
if (!messages.empty())
consumer->async_commit(*std::prev(current));
}
else
{
/// Commit everything we polled so far because either:
/// - read all polled messages (current == messages.end()),
/// - read nothing at all (messages.empty()),
/// - stalled.
consumer->async_commit();
}
PrintOffsets("Committed offset", consumer->get_offsets_committed(consumer->get_assignment()));
@ -152,6 +145,15 @@ bool ReadBufferFromKafkaConsumer::nextImpl()
if (stalled || stopped)
return false;
if (put_delimiter)
{
BufferBase::set(&delimiter, 1, 0);
put_delimiter = false;
return true;
}
put_delimiter = (delimiter != 0);
if (current == messages.end())
{
if (intermediate_commit)

View File

@ -2,15 +2,18 @@
#include <Core/Names.h>
#include <Core/Types.h>
#include <IO/DelimitedReadBuffer.h>
#include <common/logger_useful.h>
#include <IO/ReadBuffer.h>
#include <cppkafka/cppkafka.h>
namespace Poco
{
class Logger;
}
namespace DB
{
using ConsumerBufferPtr = std::shared_ptr<DelimitedReadBuffer>;
using ConsumerPtr = std::shared_ptr<cppkafka::Consumer>;
class ReadBufferFromKafkaConsumer : public ReadBuffer
@ -22,6 +25,7 @@ public:
size_t max_batch_size,
size_t poll_timeout_,
bool intermediate_commit_,
char delimiter_,
const std::atomic<bool> & stopped_);
~ReadBufferFromKafkaConsumer() override;
@ -48,6 +52,9 @@ private:
bool stalled = false;
bool intermediate_commit = true;
char delimiter;
bool put_delimiter = false;
const std::atomic<bool> & stopped;
Messages messages;
@ -56,4 +63,6 @@ private:
bool nextImpl() override;
};
using ConsumerBufferPtr = std::shared_ptr<ReadBufferFromKafkaConsumer>;
}

View File

@ -277,9 +277,7 @@ ConsumerBufferPtr StorageKafka::createReadBuffer()
size_t poll_timeout = settings.stream_poll_timeout_ms.totalMilliseconds();
/// NOTE: we pass |stream_cancelled| by reference here, so the buffers should not outlive the storage.
return std::make_shared<DelimitedReadBuffer>(
std::make_unique<ReadBufferFromKafkaConsumer>(consumer, log, batch_size, poll_timeout, intermediate_commit, stream_cancelled),
row_delimiter);
return std::make_shared<ReadBufferFromKafkaConsumer>(consumer, log, batch_size, poll_timeout, intermediate_commit, row_delimiter, stream_cancelled);
}
@ -400,7 +398,7 @@ bool StorageKafka::streamToViews()
else
in = streams[0];
std::atomic<bool> stub;
std::atomic<bool> stub = {false};
copyData(*in, *block_io.out, &stub);
// Check whether the limits were applied during query execution

View File

@ -1055,7 +1055,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
NameSet files_to_skip = {"checksums.txt", "columns.txt"};
auto mrk_extension = data_settings->index_granularity_bytes ? getAdaptiveMrkExtension() : getNonAdaptiveMrkExtension();
/// Don't change granularity type while mutating subset of columns
auto mrk_extension = source_part->index_granularity_info.is_adaptive ? getAdaptiveMrkExtension() : getNonAdaptiveMrkExtension();
for (const auto & entry : updated_header)
{
IDataType::StreamCallback callback = [&](const IDataType::SubstreamPath & substream_path)
@ -1071,7 +1072,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor
for (const auto & index : indices_to_recalc)
{
files_to_skip.insert(index->getFileName() + ".idx");
files_to_skip.insert(index->getFileName() + ".mrk");
files_to_skip.insert(index->getFileName() + mrk_extension);
}
Poco::DirectoryIterator dir_end;

View File

@ -24,6 +24,7 @@
#include <Poco/DirectoryIterator.h>
#include <Poco/File.h>
#include <optional>
#include <Interpreters/MutationsInterpreter.h>
namespace DB
@ -38,6 +39,7 @@ namespace ErrorCodes
extern const int CANNOT_ASSIGN_OPTIMIZE;
extern const int INCOMPATIBLE_COLUMNS;
extern const int UNKNOWN_SETTING;
extern const int TOO_BIG_AST;
}
namespace ActionLocks
@ -178,6 +180,7 @@ void StorageMergeTree::truncate(const ASTPtr &, const Context &, TableStructureW
LOG_INFO(log, "Removed " << parts_to_remove.size() << " parts.");
}
clearOldMutations(true);
clearOldPartsFromFilesystem();
}
@ -647,6 +650,7 @@ bool StorageMergeTree::merge(
bool StorageMergeTree::tryMutatePart()
{
auto table_lock_holder = lockStructureForShare(true, RWLockImpl::NO_QUERY);
size_t max_ast_elements = global_context.getSettingsRef().max_expanded_ast_elements;
FutureMergedMutatedPart future_part;
MutationCommands commands;
@ -671,8 +675,18 @@ bool StorageMergeTree::tryMutatePart()
if (merger_mutator.getMaxSourcePartSizeForMutation() < part->bytes_on_disk)
continue;
size_t current_ast_elements = 0;
for (auto it = mutations_begin_it; it != mutations_end_it; ++it)
{
MutationsInterpreter interpreter(shared_from_this(), it->second.commands, global_context);
size_t commands_size = interpreter.evaluateCommandsSize();
if (current_ast_elements + commands_size >= max_ast_elements)
break;
current_ast_elements += commands_size;
commands.insert(commands.end(), it->second.commands.begin(), it->second.commands.end());
}
auto new_part_info = part->info;
new_part_info.mutation = current_mutations_by_version.rbegin()->first;
@ -810,31 +824,34 @@ Int64 StorageMergeTree::getCurrentMutationVersion(
return it->first;
}
void StorageMergeTree::clearOldMutations()
void StorageMergeTree::clearOldMutations(bool truncate)
{
const auto settings = getSettings();
if (!settings->finished_mutations_to_keep)
if (!truncate && !settings->finished_mutations_to_keep)
return;
std::vector<MergeTreeMutationEntry> mutations_to_delete;
{
std::lock_guard lock(currently_merging_mutex);
if (current_mutations_by_version.size() <= settings->finished_mutations_to_keep)
if (!truncate && current_mutations_by_version.size() <= settings->finished_mutations_to_keep)
return;
auto begin_it = current_mutations_by_version.begin();
std::optional<Int64> min_version = getMinPartDataVersion();
auto end_it = current_mutations_by_version.end();
if (min_version)
auto begin_it = current_mutations_by_version.begin();
size_t to_delete_count = std::distance(begin_it, end_it);
if (!truncate)
{
if (std::optional<Int64> min_version = getMinPartDataVersion())
end_it = current_mutations_by_version.upper_bound(*min_version);
size_t done_count = std::distance(begin_it, end_it);
if (done_count <= settings->finished_mutations_to_keep)
return;
size_t to_delete_count = done_count - settings->finished_mutations_to_keep;
to_delete_count = done_count - settings->finished_mutations_to_keep;
}
auto it = begin_it;
for (size_t i = 0; i < to_delete_count; ++i)

View File

@ -116,7 +116,7 @@ private:
const DataPartPtr & part,
std::lock_guard<std::mutex> & /* currently_merging_mutex_lock */) const;
void clearOldMutations();
void clearOldMutations(bool truncate = false);
// Partition helpers
void dropPartition(const ASTPtr & partition, bool detach, const Context & context);

View File

@ -0,0 +1,12 @@
<?xml version="1.0"?>
<yandex>
<logger>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/clickhouse-server.err.log</errorlog>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
<path>/var/lib/clickhouse/</path>
<users_config>users.xml</users_config>
</yandex>

View File

@ -0,0 +1,8 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
<max_expanded_ast_elements>500</max_expanded_ast_elements>
</default>
</profiles>
</yandex>

View File

@ -0,0 +1,56 @@
from contextlib import contextmanager
import time
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
instance_test_mutations = cluster.add_instance('test_mutations_with_merge_tree', main_configs=['configs/config.xml'], user_configs=['configs/users.xml'])
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
instance_test_mutations.query('''CREATE TABLE test_mutations_with_ast_elements(date Date, a UInt64, b String) ENGINE = MergeTree(date, (a, date), 8192)''')
instance_test_mutations.query('''INSERT INTO test_mutations_with_ast_elements SELECT '2019-07-29' AS date, 1, toString(number) FROM numbers(1)''')
yield cluster
finally:
cluster.shutdown()
def test_mutations_with_merge_background_task(started_cluster):
instance_test_mutations.query('''SYSTEM STOP MERGES test_mutations_with_ast_elements''')
## The number of asts per query is 15
for execution_times_for_mutation in range(100):
instance_test_mutations.query('''ALTER TABLE test_mutations_with_ast_elements DELETE WHERE 1 = 1 AND toUInt32(b) IN (1)''')
all_done = False
for wait_times_for_mutation in range(100): # wait for replication 80 seconds max
time.sleep(0.8)
def get_done_mutations(instance):
instance_test_mutations.query('''DETACH TABLE test_mutations_with_ast_elements''')
instance_test_mutations.query('''ATTACH TABLE test_mutations_with_ast_elements''')
return int(instance.query("SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations_with_ast_elements'").rstrip())
if get_done_mutations(instance_test_mutations) == 100:
all_done = True
break
print instance_test_mutations.query("SELECT mutation_id, command, parts_to_do, is_done FROM system.mutations WHERE table = 'test_mutations_with_ast_elements' FORMAT TSVWithNames")
assert all_done
def test_mutations_with_truncate_table(started_cluster):
instance_test_mutations.query('''SYSTEM STOP MERGES test_mutations_with_ast_elements''')
## The number of asts per query is 15
for execute_number in range(100):
instance_test_mutations.query('''ALTER TABLE test_mutations_with_ast_elements DELETE WHERE 1 = 1 AND toUInt32(b) IN (1)''')
instance_test_mutations.query("TRUNCATE TABLE test_mutations_with_ast_elements")
assert instance_test_mutations.query("SELECT COUNT() FROM system.mutations WHERE table = 'test_mutations_with_ast_elements'").rstrip() == '0'

View File

@ -0,0 +1,24 @@
<?xml version="1.0"?>
<yandex>
<profiles>
<default>
<stream_poll_timeout_ms>30000</stream_poll_timeout_ms>
</default>
</profiles>
<users>
<default>
<password></password>
<networks incl="networks" replace="replace">
<ip>::/0</ip>
</networks>
<profile>default</profile>
<quota>default</quota>
</default>
</users>
<quotas>
<default>
</default>
</quotas>
</yandex>

View File

@ -30,6 +30,7 @@ import kafka_pb2
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance',
config_dir='configs',
main_configs=['configs/kafka.xml'],
with_kafka=True,
clickhouse_path_dir='clickhouse_path')
@ -136,7 +137,7 @@ def kafka_setup_teardown():
# Tests
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_settings_old_syntax(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)
@ -159,7 +160,7 @@ def test_kafka_settings_old_syntax(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_settings_new_syntax(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)
@ -195,7 +196,7 @@ def test_kafka_settings_new_syntax(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_csv_with_delimiter(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)
@ -221,7 +222,7 @@ def test_kafka_csv_with_delimiter(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_tsv_with_delimiter(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)
@ -247,7 +248,7 @@ def test_kafka_tsv_with_delimiter(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_json_without_delimiter(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)
@ -277,7 +278,7 @@ def test_kafka_json_without_delimiter(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_protobuf(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value String)
@ -302,7 +303,7 @@ def test_kafka_protobuf(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_materialized_view(kafka_cluster):
instance.query('''
DROP TABLE IF EXISTS test.view;
@ -339,7 +340,7 @@ def test_kafka_materialized_view(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_many_materialized_views(kafka_cluster):
instance.query('''
DROP TABLE IF EXISTS test.view1;
@ -437,7 +438,7 @@ def test_kafka_flush_on_big_message(kafka_cluster):
assert int(result) == kafka_messages*batch_messages, 'ClickHouse lost some messages: {}'.format(result)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_virtual_columns(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)
@ -467,7 +468,7 @@ def test_kafka_virtual_columns(kafka_cluster):
kafka_check_result(result, True, 'test_kafka_virtual1.reference')
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_virtual_columns_with_materialized_view(kafka_cluster):
instance.query('''
DROP TABLE IF EXISTS test.view;
@ -504,7 +505,7 @@ def test_kafka_virtual_columns_with_materialized_view(kafka_cluster):
kafka_check_result(result, True, 'test_kafka_virtual2.reference')
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_insert(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)
@ -541,7 +542,7 @@ def test_kafka_insert(kafka_cluster):
kafka_check_result(result, True)
@pytest.mark.timeout(60)
@pytest.mark.timeout(180)
def test_kafka_produce_consume(kafka_cluster):
instance.query('''
CREATE TABLE test.kafka (key UInt64, value UInt64)

View File

@ -1 +1 @@
SELECT number, joined FROM system.numbers ANY LEFT JOIN (SELECT number * 2 AS number, number * 10 + 1 AS joined FROM system.numbers LIMIT 10) USING number LIMIT 10
SELECT number, joined FROM system.numbers ANY LEFT JOIN (SELECT number * 2 AS number, number * 10 + 1 AS joined FROM system.numbers LIMIT 10) js2 USING number LIMIT 10

View File

@ -1,9 +1,9 @@
SELECT * FROM
(
SELECT number AS k FROM system.numbers LIMIT 10
)
) js1
ALL LEFT JOIN
(
SELECT intDiv(number, 2) AS k, number AS joined FROM system.numbers LIMIT 10
)
) js2
USING k;

View File

@ -1,9 +1,9 @@
SELECT * FROM
(
SELECT reinterpretAsString(number + reinterpretAsUInt8('A')) AS k FROM system.numbers LIMIT 10
)
) js1
ALL LEFT JOIN
(
SELECT reinterpretAsString(intDiv(number, 2) + reinterpretAsUInt8('A')) AS k, number AS joined FROM system.numbers LIMIT 10
)
) js2
USING k;

View File

@ -1,9 +1,9 @@
SELECT left, right FROM
(
SELECT number % 4 AS k1, number % 3 AS k2, number AS left FROM system.numbers LIMIT 10
)
) js1
ALL LEFT JOIN
(
SELECT number % 2 AS k1, number % 6 AS k2, number AS right FROM system.numbers LIMIT 10
)
) js2
USING k1, k2;

View File

@ -1,9 +1,9 @@
SELECT left, right FROM
(
SELECT number % 4 AS k1, toString(number % 3) AS k2, number AS left FROM system.numbers LIMIT 10
)
) js1
ALL LEFT JOIN
(
SELECT number % 2 AS k1, toString(number % 6) AS k2, number AS right FROM system.numbers LIMIT 10
)
) js2
USING k1, k2;

View File

@ -1 +1 @@
SELECT number, number / 2 AS n, j1, j2 FROM system.numbers ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) USING n LIMIT 10
SELECT number, number / 2 AS n, j1, j2 FROM system.numbers ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) js2 USING n LIMIT 10

View File

@ -1,5 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ANY INNER JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING (a) ORDER BY a,b,c,d ASC;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING (a) ORDER BY a,b,c,d ASC;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) USING a,b ORDER BY a,b,c,d ASC;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) js1 ANY INNER JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) js2 USING (a) ORDER BY a,b,c,d ASC;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) js1 ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) js2 USING (a) ORDER BY a,b,c,d ASC;
SELECT a,b,c,d FROM (SELECT 1 AS a,2 AS b, 3 AS c UNION ALL SELECT 2,3,4 ) js1 ALL LEFT JOIN (SELECT 1 AS a,2 AS b,4 AS d UNION ALL SELECT 2,3,5) js2 USING a,b ORDER BY a,b,c,d ASC;

View File

@ -11,4 +11,4 @@ ANY LEFT JOIN
SELECT
2 AS DomainID,
'def' AS Domain
) USING DomainID;
) js2 USING DomainID;

View File

@ -3,9 +3,9 @@ DROP TABLE IF EXISTS join;
CREATE TABLE join (k UInt64, s String) ENGINE = Join(ANY, LEFT, k);
INSERT INTO join VALUES (1, 'abc'), (2, 'def');
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k;
INSERT INTO join VALUES (6, 'ghi');
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k;
DROP TABLE join;

View File

@ -6,9 +6,9 @@ INSERT INTO join VALUES ('abc', [0], 1), ('def', [1, 2], 2);
INSERT INTO join (k, s) VALUES (3, 'ghi');
INSERT INTO join (x, k) VALUES ([3, 4, 5], 4);
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT s, x FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT x, s, k FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT 1, x, 2, s, 3, k, 4 FROM (SELECT number AS k FROM system.numbers LIMIT 10) ANY LEFT JOIN join USING k;
SELECT k, s FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k;
SELECT s, x FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k;
SELECT x, s, k FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k;
SELECT 1, x, 2, s, 3, k, 4 FROM (SELECT number AS k FROM system.numbers LIMIT 10) js1 ANY LEFT JOIN join USING k;
DROP TABLE join;

View File

@ -1,2 +1,6 @@
SELECT value FROM system.one ANY LEFT JOIN (SELECT dummy, dummy AS value) USING dummy GROUP BY value;
SELECT value1, value2, sum(number) FROM (SELECT number, intHash64(number) AS value1 FROM system.numbers LIMIT 10) ANY LEFT JOIN (SELECT number, intHash32(number) AS value2 FROM system.numbers LIMIT 10) USING number GROUP BY value1, value2;
SELECT value FROM system.one ANY LEFT JOIN (SELECT dummy, dummy AS value) js2 USING dummy GROUP BY value;
SELECT value1, value2, sum(number)
FROM (SELECT number, intHash64(number) AS value1 FROM system.numbers LIMIT 10) js1
ANY LEFT JOIN (SELECT number, intHash32(number) AS value2 FROM system.numbers LIMIT 10) js2
USING number GROUP BY value1, value2;

View File

@ -1 +1 @@
SELECT 1 AS k FROM system.one ANY LEFT JOIN (SELECT k FROM (SELECT 1 AS k, 2 AS x)) USING k;
SELECT 1 AS k FROM system.one ANY LEFT JOIN (SELECT k FROM (SELECT 1 AS k, 2 AS x)) js2 USING k;

View File

@ -1,3 +1,5 @@
SET joined_subquery_requires_alias = 0;
SELECT k, s1, s2 FROM (SELECT intDiv(number, 3) AS k, sum(number) AS s1 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k) ANY LEFT JOIN (SELECT intDiv(number, 4) AS k, sum(number) AS s2 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k) USING k ORDER BY k;
SELECT k, s1, s2 FROM (SELECT intDiv(number, 3) AS k, sum(number) AS s1 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k WITH TOTALS) ANY LEFT JOIN (SELECT intDiv(number, 4) AS k, sum(number) AS s2 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k) USING k ORDER BY k;
SELECT k, s1, s2 FROM (SELECT intDiv(number, 3) AS k, sum(number) AS s1 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k) ANY LEFT JOIN (SELECT intDiv(number, 4) AS k, sum(number) AS s2 FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY k WITH TOTALS) USING k ORDER BY k;

View File

@ -1 +1 @@
SELECT toFloat64(dummy + 2) AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) USING n LIMIT 10;
SELECT toFloat64(dummy + 2) AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) jr1 GLOBAL ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 10) jr2 USING n LIMIT 10;

View File

@ -1,4 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;
SELECT number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.numbers) ANY LEFT JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;
SELECT dummy + 2 AS number, number / 2 AS n, j1, j2 FROM remote('127.0.0.{2,3}', system.one) ANY INNER JOIN (SELECT number / 3 AS n, number AS j1, 'Hello' AS j2 FROM system.numbers LIMIT 0) USING n LIMIT 10;

View File

@ -8,10 +8,10 @@ FROM
arrayJoin([1, 2, 3]) AS key1,
0 AS key2,
999 AS table_1
) ALL INNER JOIN
) js1 ALL INNER JOIN
(
SELECT
arrayJoin([1, 3, 2]) AS key1,
0 AS key2,
999 AS table_1
) USING key2, key1;
) js2 USING key2, key1;

View File

@ -1,4 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS series;

View File

@ -1,4 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS series;

View File

@ -1 +1 @@
SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5);
SELECT x, y FROM (SELECT number AS x FROM system.numbers LIMIT 3) js1 CROSS JOIN (SELECT number AS y FROM system.numbers LIMIT 5) js2;

View File

@ -1,4 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;
SELECT k, x, y FROM (SELECT arrayJoin([1, 2, 3]) AS k, 'Hello' AS x) ANY FULL JOIN (SELECT range(k) AS y, arrayJoin([3, 4, 5]) AS k) USING k WHERE k < 10 ORDER BY k;
SELECT k, x FROM (SELECT arrayJoin([1, 2, 3]) AS k, 'Hello' AS x) ANY FULL JOIN (SELECT range(k) AS y, arrayJoin([3, 4, 5]) AS k) USING k WHERE k < 10 ORDER BY k;

View File

@ -1,2 +1,4 @@
SET joined_subquery_requires_alias = 0;
SELECT k, a FROM (SELECT 42 AS k FROM remote('127.0.0.2', system.one)) GLOBAL ALL FULL OUTER JOIN (SELECT 42 AS k, 1 AS a, a) USING k;
SELECT 1 FROM remote('127.0.0.2', system.one) WHERE (1, 1) GLOBAL IN (SELECT 1 AS a, a);

View File

@ -1,8 +1,8 @@
SET any_join_distinct_right_table_keys = 1;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING a, b;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING b, a;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) js2 USING a, b;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) js2 USING b, a;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) USING a, b;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) USING b, a;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) js2 USING a, b;
SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) js1 ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) js2 USING b, a;

View File

@ -1,9 +1,9 @@
select b from (select 1 as a, 42 as c) any left join (select 2 as b, 2 as b, 41 as c) using c;
select b from (select 1 as a, 42 as c) any left join (select 2 as b, 2 as b, 42 as c) using c;
select b from (select 1 as a, 42 as c) js1 any left join (select 2 as b, 2 as b, 41 as c) js2 using c;
select b from (select 1 as a, 42 as c) js1 any left join (select 2 as b, 2 as b, 42 as c) js2 using c;
select c,a,a,b,b from
(select 1 as a, 1 as a, 42 as c group by c order by a,c)
(select 1 as a, 1 as a, 42 as c group by c order by a,c) js1
any left join
(select 2 as b, 2 as b, 41 as c group by c order by b,c)
(select 2 as b, 2 as b, 41 as c group by c order by b,c) js2
using c
order by b;

View File

@ -6,8 +6,8 @@ FROM
(
SELECT 6534090703218709881 AS DeviceIDHash, 1458586663 AS InstallTimestamp
UNION ALL SELECT 2697418689476658272, 1458561552
) ANY INNER JOIN
) js1 ANY INNER JOIN
(
SELECT 1034415739529768519 AS DeviceIDHash, 1458566664 AS ReferrerTimestamp
UNION ALL SELECT 2697418689476658272, 1458561552
) USING DeviceIDHash;
) js2 USING DeviceIDHash;

View File

@ -1,2 +1,2 @@
set any_join_distinct_right_table_keys = 1;
select a from (select (1, 2) as a) any inner join (select (1, 2) as a) using a;
select a from (select (1, 2) as a) js1 any inner join (select (1, 2) as a) js2 using a;

View File

@ -1,4 +1,5 @@
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;
select x, y from (select 1 as x, 2 as y, x, y);
select x, y from (select 1 as x, 1 as y, x, y);

View File

@ -12,7 +12,7 @@ FROM
UNION ALL SELECT 'aaaaaaaabaaaaaaa'
UNION ALL SELECT 'aaaaaaabaaaaaaaa'
UNION ALL SELECT 'aaaaaaacaaaaaaaa'
)
) js1
CROSS JOIN
(
SELECT 'aaaaaaaaaaaaaaaa' AS b
@ -24,7 +24,7 @@ CROSS JOIN
UNION ALL SELECT 'aaaaaaaabaaaaaaa'
UNION ALL SELECT 'aaaaaaabaaaaaaaa'
UNION ALL SELECT 'aaaaaaacaaaaaaaa'
)
) js2
ORDER BY a, b;

View File

@ -1,5 +1,6 @@
SET join_use_nulls = 0;
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;
SELECT k, a, b
FROM

View File

@ -5,28 +5,28 @@ SELECT k, a, b
FROM
(
SELECT nullIf(number, 7) AS k, toString(number) AS a FROM system.numbers LIMIT 10
)
) js1
ANY INNER JOIN
(
SELECT number AS k, toString(number) AS b FROM system.numbers LIMIT 5, 10
) USING (k) ORDER BY k;
) js2 USING (k) ORDER BY k;
SELECT k, a, b
FROM
(
SELECT number AS k, toString(number) AS a FROM system.numbers LIMIT 10
)
) js1
ANY LEFT JOIN
(
SELECT nullIf(number, 8) AS k, toString(number) AS b FROM system.numbers LIMIT 5, 10
) USING (k) ORDER BY k;
) js2 USING (k) ORDER BY k;
SELECT k, a, b
FROM
(
SELECT nullIf(number, 7) AS k, toString(number) AS a FROM system.numbers LIMIT 10
)
) js1
ANY RIGHT JOIN
(
SELECT nullIf(number, 8) AS k, toString(number) AS b FROM system.numbers LIMIT 5, 10
) USING (k) ORDER BY k;
) js2 USING (k) ORDER BY k;

View File

@ -13,11 +13,11 @@ DROP TABLE set;
DROP TABLE IF EXISTS join;
CREATE TABLE join (k UInt8, x String) ENGINE = Memory;
INSERT INTO join VALUES (1, 'hello');
SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) ANY LEFT JOIN join USING k;
SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) js1 ANY LEFT JOIN join USING k;
DROP TABLE join;
CREATE TABLE join (k UInt8, x String) ENGINE = Join(ANY, LEFT, k);
INSERT INTO join VALUES (1, 'hello');
SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) ANY LEFT JOIN join USING k;
SELECT k, x FROM (SELECT arrayJoin([1, 2]) AS k) js1 ANY LEFT JOIN join USING k;
DROP TABLE join;

View File

@ -14,7 +14,7 @@ FROM
toString(number) AS a
FROM system.numbers
LIMIT 2
)
) js1
ANY LEFT JOIN
(
SELECT
@ -22,7 +22,7 @@ ANY LEFT JOIN
toString(number) AS b
FROM system.numbers
LIMIT 1, 2
) USING (k)
) js2 USING (k)
ORDER BY k ASC;
SELECT * FROM null_00549 ORDER BY k, a, b;

View File

@ -1,23 +1,24 @@
0 1 0 1 0 1 1 1
\N 0 1 0 0 1 0 1 1 1
\N 1 \N 1 \N 1 1 1
0 \N 0 \N 0 \N \N 0
\N \N 1 \N \N 1 \N 1 1 1
1 1 1 1 1 1 1 1 1 1
\N 0 1 0 0 1 0 1 1 1
\N \N 1 \N \N 1 \N 1 1 1
0 0 1 0 0 1 0 1 1 1
\N 0 1 0 0 1 0 1 1 1
\N \N 1 \N \N 1 \N 1 1 1
2 1 1 1 1 1 1 1 1 1
\N 0 1 0 0 1 0 1 1 1
\N \N 1 \N \N 1 \N 1 1 1
1 1 1 1 1 1 1 1 1 1
\N 0 1 0 0 1 0 1 1 1
\N \N 1 \N \N 1 \N 1 1 1
0 0 1 0 0 1 0 1 1 1
\N \N \N \N \N \N \N \N \N \N
\N 0 \N \N 0 \N 0 \N \N 0
1 0 1 1 0 1 0 1 1 0
\N \N \N \N \N \N \N \N \N \N
\N 0 \N \N 0 \N 0 \N \N 0
0 0 0 0 0 0 0 0 0 0
\N \N \N \N \N \N \N \N \N \N
2 0 2 2 0 2 0 2 2 0
\N \N \N \N \N \N \N \N \N \N
\N 0 \N \N 0 \N 0 \N \N 0
2 0 1 1 0 1 0 1 1 0
\N 0 \N \N 0 \N 0 \N \N 0
1 0 1 1 0 1 0 1 1 0
\N \N \N \N \N \N \N \N \N \N
\N 0 \N \N 0 \N 0 \N \N 0
0 0 0 0 0 0 0 0 0 0
1
0

View File

@ -8,6 +8,16 @@ SELECT
NULL OR 1 OR NULL,
0 OR NULL OR 1 OR NULL;
SELECT
0 AND NULL,
1 AND NULL,
toNullable(0) AND NULL,
toNullable(1) AND NULL,
0.0 AND NULL,
0.1 AND NULL,
NULL AND 1 AND NULL,
0 AND NULL AND 1 AND NULL;
SELECT
x,
0 OR x,

View File

@ -1,3 +1,3 @@
SET any_join_distinct_right_table_keys = 1;
SET max_block_size = 10;
SELECT * FROM (select toUInt64(1) s limit 1) any right join (select number s, s as x from numbers(11)) using (s) ORDER BY s;
SELECT * FROM (select toUInt64(1) s limit 1) js1 any right join (select number s, s as x from numbers(11)) js2 using (s) ORDER BY s;

View File

@ -35,7 +35,7 @@ from joinbug;
select id, id2, val, val2, created
from (
SELECT toUInt64(arrayJoin(range(50))) AS id2
)
) js1
ANY INNER JOIN joinbug_join using id2;
DROP TABLE joinbug;

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