Merge branch 'merging-aggregated-processors' into processors-2

This commit is contained in:
Nikolai Kochetov 2019-09-06 15:21:17 +03:00
commit 52305cefbc
272 changed files with 7434 additions and 3842 deletions

1
.gitignore vendored
View File

@ -90,7 +90,6 @@ dbms/src/Core/tests/field
dbms/src/Core/tests/rvo_test
dbms/src/Core/tests/string_pool
dbms/src/DataStreams/tests/aggregating_stream
dbms/src/DataStreams/tests/block_row_transforms
dbms/src/DataStreams/tests/block_tab_separated_streams
dbms/src/DataStreams/tests/collapsing_sorted_stream
dbms/src/DataStreams/tests/expression_stream

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

@ -21,6 +21,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <common/phdr_cache.h>
#include <ext/scope_guard.h>
/// Universal executable for various clickhouse applications
@ -130,8 +131,19 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
}
/// This allows to implement assert to forbid initialization of a class in static constructors.
/// Usage:
///
/// extern bool inside_main;
/// class C { C() { assert(inside_main); } };
bool inside_main = false;
int main(int argc_, char ** argv_)
{
inside_main = true;
SCOPE_EXIT({ inside_main = false; });
/// Reset new handler to default (that throws std::bad_alloc)
/// It is needed because LLVM library clobbers it.
std::set_new_handler(nullptr);

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

@ -636,6 +636,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)
{
@ -725,6 +732,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);
@ -747,19 +771,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:
@ -771,12 +795,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;
@ -845,6 +868,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()
{
@ -883,6 +926,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()
{
@ -897,6 +961,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,12 @@
<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>
</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

@ -447,6 +447,8 @@ namespace ErrorCodes
extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470;
extern const int SETTINGS_ARE_NOT_SUPPORTED = 471;
extern const int READONLY_SETTING = 472;
extern const int DEADLOCK_AVOIDED = 473;
extern const int INVALID_TEMPLATE_FORMAT = 474;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

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

@ -4,6 +4,8 @@
#include <Common/CurrentMetrics.h>
#include <Common/ProfileEvents.h>
#include <cassert>
namespace ProfileEvents
{
@ -29,6 +31,7 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int DEADLOCK_AVOIDED;
}
@ -52,6 +55,44 @@ public:
};
namespace
{
/// Global information about all read locks that query has. It is needed to avoid some type of deadlocks.
class QueryLockInfo
{
private:
std::mutex mutex;
std::map<std::string, size_t> queries;
public:
void add(const String & query_id)
{
std::lock_guard lock(mutex);
++queries[query_id];
}
void remove(const String & query_id)
{
std::lock_guard lock(mutex);
auto it = queries.find(query_id);
assert(it != queries.end());
if (--it->second == 0)
queries.erase(it);
}
void check(const String & query_id)
{
std::lock_guard lock(mutex);
if (queries.count(query_id))
throw Exception("Possible deadlock avoided. Client should retry.", ErrorCodes::DEADLOCK_AVOIDED);
}
};
QueryLockInfo all_read_locks;
}
RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id)
{
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
@ -68,29 +109,48 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
GroupsContainer::iterator it_group;
ClientsContainer::iterator it_client;
/// This object is placed above unique_lock, because it may lock in destructor.
LockHolder res;
std::unique_lock lock(mutex);
/// Check if the same query is acquiring previously acquired lock
LockHolder existing_holder_ptr;
if (query_id != RWLockImpl::NO_QUERY)
{
auto it_query = query_id_to_holder.find(query_id);
if (it_query != query_id_to_holder.end())
existing_holder_ptr = it_query->second.lock();
res = it_query->second.lock();
}
if (existing_holder_ptr)
if (res)
{
/// XXX: it means we can't upgrade lock from read to write - with proper waiting!
if (type != Read || existing_holder_ptr->it_group->type != Read)
if (type != Read || res->it_group->type != Read)
throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR);
return existing_holder_ptr;
else
return res;
}
/** If the query already has any active read lock and tries to acquire another read lock
* but it is not in front of the queue and has to wait, deadlock is possible:
*
* Example (four queries, two RWLocks - 'a' and 'b'):
*
* --> time -->
*
* q1: ra rb
* q2: wa
* q3: rb ra
* q4: wb
*
* We will throw an exception instead.
*/
if (type == Type::Write || queue.empty() || queue.back().type == Type::Write)
{
if (type == Type::Read && !queue.empty() && queue.back().type == Type::Write && query_id != RWLockImpl::NO_QUERY)
all_read_locks.check(query_id);
/// Create new group of clients
it_group = queue.emplace(queue.end(), type);
}
@ -98,6 +158,9 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
{
/// Will append myself to last group
it_group = std::prev(queue.end());
if (it_group != queue.begin() && query_id != RWLockImpl::NO_QUERY)
all_read_locks.check(query_id);
}
/// Append myself to the end of chosen group
@ -114,14 +177,19 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
throw;
}
LockHolder res(new LockHolderImpl(shared_from_this(), it_group, it_client));
res.reset(new LockHolderImpl(shared_from_this(), it_group, it_client));
/// Wait a notification until we will be the only in the group.
it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); });
/// Insert myself (weak_ptr to the holder) to queries set to implement recursive lock
if (query_id != RWLockImpl::NO_QUERY)
{
query_id_to_holder.emplace(query_id, res);
if (type == Type::Read)
all_read_locks.add(query_id);
}
res->query_id = query_id;
finalize_metrics();
@ -131,11 +199,14 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String &
RWLockImpl::LockHolderImpl::~LockHolderImpl()
{
std::unique_lock lock(parent->mutex);
std::lock_guard lock(parent->mutex);
/// Remove weak_ptrs to the holder, since there are no owners of the current lock
parent->query_id_to_holder.erase(query_id);
if (*it_client == RWLockImpl::Read && query_id != RWLockImpl::NO_QUERY)
all_read_locks.remove(query_id);
/// Removes myself from client list of our group
it_group->clients.erase(it_client);
@ -156,6 +227,7 @@ RWLockImpl::LockHolderImpl::LockHolderImpl(RWLock && parent_, RWLockImpl::Groups
: parent{std::move(parent_)}, it_group{it_group_}, it_client{it_client_},
active_client_increment{(*it_client == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders
: CurrentMetrics::RWLockActiveWriters}
{}
{
}
}

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

@ -13,6 +13,14 @@
using namespace DB;
namespace DB
{
namespace ErrorCodes
{
extern const int DEADLOCK_AVOIDED;
}
}
TEST(Common, RWLock_1)
{
@ -123,6 +131,74 @@ TEST(Common, RWLock_Recursive)
}
TEST(Common, RWLock_Deadlock)
{
static auto lock1 = RWLockImpl::create();
static auto lock2 = RWLockImpl::create();
/**
* q1: r1 r2
* q2: w1
* q3: r2 r1
* q4: w2
*/
std::thread t1([&] ()
{
auto holder1 = lock1->getLock(RWLockImpl::Read, "q1");
usleep(100000);
usleep(100000);
usleep(100000);
try
{
auto holder2 = lock2->getLock(RWLockImpl::Read, "q1");
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::DEADLOCK_AVOIDED)
throw;
}
});
std::thread t2([&] ()
{
usleep(100000);
auto holder1 = lock1->getLock(RWLockImpl::Write, "q2");
});
std::thread t3([&] ()
{
usleep(100000);
usleep(100000);
auto holder2 = lock2->getLock(RWLockImpl::Read, "q3");
usleep(100000);
usleep(100000);
try
{
auto holder1 = lock1->getLock(RWLockImpl::Read, "q3");
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::DEADLOCK_AVOIDED)
throw;
}
});
std::thread t4([&] ()
{
usleep(100000);
usleep(100000);
usleep(100000);
auto holder2 = lock2->getLock(RWLockImpl::Write, "q4");
});
t1.join();
t2.join();
t3.join();
t4.join();
}
TEST(Common, RWLock_PerfTest_Readers)
{
constexpr int cycles = 100000; // 100k

View File

@ -0,0 +1,228 @@
#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 (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 (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 (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 (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.") \
@ -216,12 +216,15 @@ struct Settings : public SettingsCollection<Settings>
M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.") \
M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.") \
M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)") \
M(SettingString, format_schema_rows, "", "Row format string for Template format") \
M(SettingString, format_schema_rows_between_delimiter, "\n", "Delimiter between rows for Template format") \
M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.") \
M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \
M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout") \
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

@ -1,178 +0,0 @@
#include <Common/Exception.h>
#include <IO/WriteHelpers.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int CANNOT_PARSE_DATE;
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
extern const int CANNOT_PARSE_NUMBER;
extern const int CANNOT_PARSE_UUID;
extern const int TOO_LARGE_STRING_SIZE;
extern const int CANNOT_READ_ALL_DATA;
extern const int INCORRECT_DATA;
extern const int INCORRECT_NUMBER_OF_COLUMNS;
}
BlockInputStreamFromRowInputStream::BlockInputStreamFromRowInputStream(
const RowInputStreamPtr & row_input_,
const Block & sample_,
UInt64 max_block_size_,
UInt64 rows_portion_size_,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
: row_input(row_input_)
, sample(sample_)
, max_block_size(max_block_size_)
, rows_portion_size(rows_portion_size_)
, read_virtual_columns_callback(callback)
, allow_errors_num(settings.input_allow_errors_num)
, allow_errors_ratio(settings.input_allow_errors_ratio)
{
}
static bool isParseError(int code)
{
return code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED
|| code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING
|| code == ErrorCodes::CANNOT_PARSE_DATE
|| code == ErrorCodes::CANNOT_PARSE_DATETIME
|| code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT
|| code == ErrorCodes::CANNOT_PARSE_NUMBER
|| code == ErrorCodes::CANNOT_PARSE_UUID
|| code == ErrorCodes::TOO_LARGE_STRING_SIZE
|| code == ErrorCodes::CANNOT_READ_ALL_DATA
|| code == ErrorCodes::INCORRECT_DATA;
}
Block BlockInputStreamFromRowInputStream::readImpl()
{
size_t num_columns = sample.columns();
MutableColumns columns = sample.cloneEmptyColumns();
block_missing_values.clear();
try
{
for (size_t rows = 0, batch = 0; rows < max_block_size; ++rows, ++batch)
{
if (rows_portion_size && batch == rows_portion_size)
{
batch = 0;
if (!checkTimeLimit() || isCancelled())
break;
}
try
{
++total_rows;
RowReadExtension info_;
if (!row_input->read(columns, info_))
break;
if (read_virtual_columns_callback)
read_virtual_columns_callback();
for (size_t column_idx = 0; column_idx < info_.read_columns.size(); ++column_idx)
{
if (!info_.read_columns[column_idx])
{
size_t column_size = columns[column_idx]->size();
if (column_size == 0)
throw Exception("Unexpected empty column", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
block_missing_values.setBit(column_idx, column_size - 1);
}
}
}
catch (Exception & e)
{
/// Logic for possible skipping of errors.
if (!isParseError(e.code()))
throw;
if (allow_errors_num == 0 && allow_errors_ratio == 0)
throw;
++num_errors;
Float32 current_error_ratio = static_cast<Float32>(num_errors) / total_rows;
if (num_errors > allow_errors_num
&& current_error_ratio > allow_errors_ratio)
{
e.addMessage("(Already have " + toString(num_errors) + " errors"
" out of " + toString(total_rows) + " rows"
", which is " + toString(current_error_ratio) + " of all rows)");
throw;
}
if (!row_input->allowSyncAfterError())
{
e.addMessage("(Input format doesn't allow to skip errors)");
throw;
}
row_input->syncAfterError();
/// Truncate all columns in block to minimal size (remove values, that was appended to only part of columns).
size_t min_size = std::numeric_limits<size_t>::max();
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
min_size = std::min(min_size, columns[column_idx]->size());
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
{
auto & column = columns[column_idx];
if (column->size() > min_size)
column->popBack(column->size() - min_size);
}
}
}
}
catch (Exception & e)
{
if (!isParseError(e.code()))
throw;
String verbose_diagnostic;
try
{
verbose_diagnostic = row_input->getDiagnosticInfo();
}
catch (...)
{
/// Error while trying to obtain verbose diagnostic. Ok to ignore.
}
e.addMessage("(at row " + toString(total_rows) + ")\n" + verbose_diagnostic);
throw;
}
if (columns.empty() || columns[0]->empty())
return {};
return sample.cloneWithColumns(std::move(columns));
}
void BlockInputStreamFromRowInputStream::readSuffix()
{
if (allow_errors_num > 0 || allow_errors_ratio > 0)
{
Logger * log = &Logger::get("BlockInputStreamFromRowInputStream");
LOG_TRACE(log, "Skipped " << num_errors << " rows with errors while reading the input stream");
}
row_input->readSuffix();
}
}

View File

@ -1,62 +0,0 @@
#pragma once
#include <Core/Defines.h>
#include <DataStreams/IBlockInputStream.h>
#include <Formats/FormatFactory.h>
#include <Formats/FormatSettings.h>
#include <Formats/IRowInputStream.h>
namespace DB
{
/** Makes block-oriented stream on top of row-oriented stream.
* It is used to read data from text formats.
*
* Also controls over parsing errors and prints diagnostic information about them.
*/
class BlockInputStreamFromRowInputStream : public IBlockInputStream
{
public:
/// |sample| is a block with zero rows, that structure describes how to interpret values
/// |rows_portion_size| is a number of rows to read before break and check limits
BlockInputStreamFromRowInputStream(
const RowInputStreamPtr & row_input_,
const Block & sample_,
UInt64 max_block_size_,
UInt64 rows_portion_size_,
FormatFactory::ReadCallback callback,
const FormatSettings & settings);
void readPrefix() override { row_input->readPrefix(); }
void readSuffix() override;
String getName() const override { return "BlockInputStreamFromRowInputStream"; }
RowInputStreamPtr & getRowInput() { return row_input; }
Block getHeader() const override { return sample; }
const BlockMissingValues & getMissingValues() const override { return block_missing_values; }
protected:
Block readImpl() override;
private:
RowInputStreamPtr row_input;
Block sample;
UInt64 max_block_size;
UInt64 rows_portion_size;
/// Callback used to setup virtual columns after reading each row.
FormatFactory::ReadCallback read_virtual_columns_callback;
BlockMissingValues block_missing_values;
UInt64 allow_errors_num;
Float32 allow_errors_ratio;
size_t total_rows = 0;
size_t num_errors = 0;
};
}

View File

@ -1,573 +0,0 @@
#include <Core/Defines.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <Formats/verbosePrintString.h>
#include <Formats/CSVRowInputStream.h>
#include <Formats/FormatFactory.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
}
static inline void skipEndOfLine(ReadBuffer & istr)
{
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
if (*istr.position() == '\n')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\r')
++istr.position();
}
else if (*istr.position() == '\r')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\n')
++istr.position();
else
throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)."
" Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA);
}
else if (!istr.eof())
throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA);
}
static inline void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column)
{
if (is_last_column)
{
if (istr.eof())
return;
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
{
++istr.position();
if (istr.eof())
return;
}
skipEndOfLine(istr);
}
else
assertChar(delimiter, istr);
}
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
{
while (!buf.eof()
&& (*buf.position() == ' '
|| *buf.position() == '\t'))
++buf.position();
}
static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns)
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(istr);
readCSVString(tmp, istr, settings);
skipWhitespacesAndTabs(istr);
skipDelimiter(istr, settings.delimiter, i + 1 == num_columns);
}
}
CSVRowInputStream::CSVRowInputStream(ReadBuffer & istr_, const Block & header_, bool with_names_, const FormatSettings & format_settings_)
: istr(istr_), header(header_), with_names(with_names_), format_settings(format_settings_)
{
const auto num_columns = header.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
column_idx_to_nullable_column_idx.resize(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = header.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
/// If input_format_null_as_default=1 we need ColumnNullable of type DataTypeNullable(nested_type)
/// to parse value as nullable before inserting it in corresponding column of not-nullable type.
/// Constructing temporary column for each row is slow, so we prepare it here
if (format_settings.csv.null_as_default && !column_info.type->isNullable() && column_info.type->canBeInsideNullable())
{
column_idx_to_nullable_column_idx[i] = nullable_columns.size();
nullable_types.emplace_back(std::make_shared<DataTypeNullable>(column_info.type));
nullable_columns.emplace_back(nullable_types.back()->createColumn());
}
}
}
/// Map an input file column to a table column, based on its name.
void CSVRowInputStream::addInputColumn(const String & column_name)
{
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
"Unknown field found in CSV header: '" + column_name + "' " +
"at position " + std::to_string(column_indexes_for_input_fields.size()) +
"\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
ErrorCodes::INCORRECT_DATA
);
}
const auto column_index = column_it->second;
if (read_columns[column_index])
throw Exception("Duplicate field found while parsing CSV header: " + column_name, ErrorCodes::INCORRECT_DATA);
read_columns[column_index] = true;
column_indexes_for_input_fields.emplace_back(column_index);
}
void CSVRowInputStream::readPrefix()
{
/// In this format, we assume, that if first string field contain BOM as value, it will be written in quotes,
/// so BOM at beginning of stream cannot be confused with BOM in first string value, and it is safe to skip it.
skipBOMIfExists(istr);
if (with_names)
{
/// This CSV file has a header row with column names. Depending on the
/// settings, use it or skip it.
if (format_settings.with_names_use_header)
{
/// Look at the file header to see which columns we have there.
/// The missing columns are filled with defaults.
read_columns.assign(header.columns(), false);
do
{
String column_name;
skipWhitespacesAndTabs(istr);
readCSVString(column_name, istr, format_settings.csv);
skipWhitespacesAndTabs(istr);
addInputColumn(column_name);
}
while (checkChar(format_settings.csv.delimiter, istr));
skipDelimiter(istr, format_settings.csv.delimiter, true);
for (size_t column = 0; column < read_columns.size(); column++)
{
if (!read_columns[column])
{
have_always_default_columns = true;
break;
}
}
return;
}
else
{
skipRow(istr, format_settings.csv, header.columns());
}
}
/// The default: map each column of the file to the column of the table with
/// the same index.
read_columns.assign(header.columns(), true);
column_indexes_for_input_fields.resize(header.columns());
for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i)
{
column_indexes_for_input_fields[i] = i;
}
}
/** If you change this function, don't forget to change its counterpart
* with extended error reporting: parseRowAndPrintDiagnosticInfo().
*/
bool CSVRowInputStream::read(MutableColumns & columns, RowReadExtension & ext)
{
if (istr.eof())
return false;
updateDiagnosticInfo();
/// Track whether we have to fill any columns in this row with default
/// values. If not, we return an empty column mask to the caller, so that
/// it doesn't have to check it.
bool have_default_columns = have_always_default_columns;
const auto delimiter = format_settings.csv.delimiter;
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
const auto & table_column = column_indexes_for_input_fields[file_column];
const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size();
if (table_column)
{
skipWhitespacesAndTabs(istr);
read_columns[*table_column] = readField(*columns[*table_column], data_types[*table_column],
is_last_file_column, *table_column);
if (!read_columns[*table_column])
have_default_columns = true;
skipWhitespacesAndTabs(istr);
}
else
{
/// We never read this column from the file, just skip it.
String tmp;
readCSVString(tmp, istr, format_settings.csv);
}
skipDelimiter(istr, delimiter, is_last_file_column);
}
if (have_default_columns)
{
for (size_t i = 0; i < read_columns.size(); i++)
{
if (!read_columns[i])
{
/// The column value for this row is going to be overwritten
/// with default by the caller, but the general assumption is
/// that the column size increases for each row, so we have
/// to insert something. Since we do not care about the exact
/// value, we do not have to use the default value specified by
/// the data type, and can just use IColumn::insertDefault().
columns[i]->insertDefault();
}
}
ext.read_columns = read_columns;
}
return true;
}
String CSVRowInputStream::getDiagnosticInfo()
{
if (istr.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
istr.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
istr.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
/** gcc-7 generates wrong code with optimization level greater than 1.
* See tests: dbms/src/IO/tests/write_int.cpp
* and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh
* This is compiler bug. The bug does not present in gcc-8 and clang-8.
* Nevertheless, we don't need high optimization of this function.
*/
bool OPTIMIZE(1) CSVRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
const char delimiter = format_settings.csv.delimiter;
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
if (file_column == 0 && istr.eof())
{
out << "<End of stream>\n";
return false;
}
if (column_indexes_for_input_fields[file_column].has_value())
{
const auto & table_column = *column_indexes_for_input_fields[file_column];
const auto & current_column_type = data_types[table_column];
const bool is_last_file_column =
file_column + 1 == column_indexes_for_input_fields.size();
const bool at_delimiter = !istr.eof() && *istr.position() == delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (istr.eof() || *istr.position() == '\n' || *istr.position() == '\r');
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(table_column).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(table_column).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
if (format_settings.csv.empty_as_default
&& (at_delimiter || at_last_column_line_end))
{
columns[table_column]->insertDefault();
}
else
{
BufferBase::Position prev_position = istr.position();
BufferBase::Position curr_position = istr.position();
std::exception_ptr exception;
try
{
skipWhitespacesAndTabs(istr);
prev_position = istr.position();
readField(*columns[table_column], current_column_type, is_last_file_column, table_column);
curr_position = istr.position();
skipWhitespacesAndTabs(istr);
}
catch (...)
{
exception = std::current_exception();
}
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, istr.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue()
&& *curr_position != '\n' && *curr_position != '\r'
&& *curr_position != delimiter)
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, istr.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
String tmp;
readCSVString(tmp, istr, format_settings.csv);
}
/// Delimiters
if (file_column + 1 == column_indexes_for_input_fields.size())
{
if (istr.eof())
return false;
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
{
++istr.position();
if (istr.eof())
break;
}
if (!istr.eof() && *istr.position() != '\n' && *istr.position() != '\r')
{
out << "ERROR: There is no line feed. ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n"
" It's like your file has more columns than expected.\n"
"And if your file have right number of columns, maybe it have unquoted string value with comma.\n";
return false;
}
skipEndOfLine(istr);
}
else
{
try
{
assertChar(delimiter, istr);
}
catch (const DB::Exception &)
{
if (*istr.position() == '\n' || *istr.position() == '\r')
{
out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped quotes in values.\n";
}
else
{
out << "ERROR: There is no delimiter (" << delimiter << "). ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
return true;
}
void CSVRowInputStream::syncAfterError()
{
skipToNextLineOrEOF(istr);
}
void CSVRowInputStream::updateDiagnosticInfo()
{
++row_num;
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = istr.count() - istr.offset();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = istr.position();
}
bool CSVRowInputStream::readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx)
{
const bool at_delimiter = !istr.eof() || *istr.position() == format_settings.csv.delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (istr.eof() || *istr.position() == '\n' || *istr.position() == '\r');
if (format_settings.csv.empty_as_default
&& (at_delimiter || at_last_column_line_end))
{
/// Treat empty unquoted column value as default value, if
/// specified in the settings. Tuple columns might seem
/// problematic, because they are never quoted but still contain
/// commas, which might be also used as delimiters. However,
/// they do not contain empty unquoted fields, so this check
/// works for tuples as well.
return false;
}
else if (column_idx_to_nullable_column_idx[column_idx])
{
/// If value is null but type is not nullable then use default value instead.
const size_t nullable_idx = *column_idx_to_nullable_column_idx[column_idx];
auto & tmp_col = *nullable_columns[nullable_idx];
nullable_types[nullable_idx]->deserializeAsTextCSV(tmp_col, istr, format_settings);
Field value = tmp_col[0];
tmp_col.popBack(1); /// do not store copy of values in memory
if (value.isNull())
return false;
column.insert(value);
return true;
}
else
{
/// Read the column normally.
type->deserializeAsTextCSV(column, istr, format_settings);
return true;
}
}
void registerInputFormatCSV(FormatFactory & factory)
{
for (bool with_names : {false, true})
{
factory.registerInputFormat(with_names ? "CSVWithNames" : "CSV", [=](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<CSVRowInputStream>(buf, sample, with_names, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
}
}

View File

@ -1,83 +0,0 @@
#pragma once
#include <optional>
#include <unordered_map>
#include <Core/Block.h>
#include <Formats/IRowInputStream.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class ReadBuffer;
/** A stream for inputting data in csv format.
* Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values.
*/
class CSVRowInputStream : public IRowInputStream
{
public:
/** with_names - in the first line the header with column names
*/
CSVRowInputStream(ReadBuffer & istr_, const Block & header_, bool with_names_, const FormatSettings & format_settings_);
bool read(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
ReadBuffer & istr;
Block header;
bool with_names;
DataTypes data_types;
const FormatSettings format_settings;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
/// Maps indexes of columns in the input file to indexes of table columns
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
/// Tracks which colums we have read in a single read() call.
/// For columns that are never read, it is initialized to false when we
/// read the file header, and never changed afterwards.
/// For other columns, it is updated on each read() call.
std::vector<UInt8> read_columns;
/// Whether we have any columns that are not read from file at all,
/// and must be always initialized with defaults.
bool have_always_default_columns = false;
void addInputColumn(const String & column_name);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those that are still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
/// For setting input_format_null_as_default
DataTypes nullable_types;
MutableColumns nullable_columns;
OptionalIndexes column_idx_to_nullable_column_idx;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
bool readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx);
};
}

View File

@ -47,6 +47,9 @@ static FormatSettings getInputFormatSetting(const Settings & settings)
format_settings.date_time_input_format = settings.date_time_input_format;
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
format_settings.template_settings.format = settings.format_schema;
format_settings.template_settings.row_format = settings.format_schema_rows;
format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter;
return format_settings;
}
@ -63,6 +66,9 @@ static FormatSettings getOutputFormatSetting(const Settings & settings)
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width;
format_settings.pretty.color = settings.output_format_pretty_color;
format_settings.template_settings.format = settings.format_schema;
format_settings.template_settings.row_format = settings.format_schema_rows;
format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter;
format_settings.write_statistics = settings.output_format_write_statistics;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
@ -220,8 +226,6 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro
void registerInputFormatNative(FormatFactory & factory);
void registerOutputFormatNative(FormatFactory & factory);
void registerInputFormatTabSeparated(FormatFactory & factory);
void registerInputFormatCSV(FormatFactory & factory);
void registerInputFormatProcessorNative(FormatFactory & factory);
void registerOutputFormatProcessorNative(FormatFactory & factory);
@ -242,6 +246,8 @@ void registerInputFormatProcessorORC(FormatFactory & factory);
void registerOutputFormatProcessorParquet(FormatFactory & factory);
void registerInputFormatProcessorProtobuf(FormatFactory & factory);
void registerOutputFormatProcessorProtobuf(FormatFactory & factory);
void registerInputFormatProcessorTemplate(FormatFactory & factory);
void registerOutputFormatProcessorTemplate(FormatFactory &factory);
/// Output only (presentational) formats.
@ -267,8 +273,6 @@ FormatFactory::FormatFactory()
{
registerInputFormatNative(*this);
registerOutputFormatNative(*this);
registerInputFormatTabSeparated(*this);
registerInputFormatCSV(*this);
registerOutputFormatProcessorJSONEachRowWithProgress(*this);
@ -292,6 +296,8 @@ FormatFactory::FormatFactory()
registerInputFormatProcessorORC(*this);
registerInputFormatProcessorParquet(*this);
registerOutputFormatProcessorParquet(*this);
registerInputFormatProcessorTemplate(*this);
registerOutputFormatProcessorTemplate(*this);
registerOutputFormatNull(*this);

View File

@ -50,6 +50,15 @@ struct FormatSettings
Values values;
struct Template
{
String format;
String row_format;
String row_between_delimiter;
};
Template template_settings;
bool skip_unknown_fields = false;
bool with_names_use_header = false;
bool write_statistics = true;

View File

@ -0,0 +1,217 @@
#include <Formats/ParsedTemplateFormatString.h>
#include <Formats/verbosePrintString.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/Operators.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_TEMPLATE_FORMAT;
}
ParsedTemplateFormatString::ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name)
{
try
{
parse(format_string, idx_by_name);
}
catch (DB::Exception & e)
{
if (e.code() != ErrorCodes::INVALID_TEMPLATE_FORMAT)
throwInvalidFormat(e.message(), columnsCount());
else
throw;
}
}
void ParsedTemplateFormatString::parse(const String & format_string, const ColumnIdxGetter & idx_by_name)
{
enum ParserState
{
Delimiter,
Column,
Format
};
const char * pos = format_string.c_str();
const char * end = format_string.c_str() + format_string.size();
const char * token_begin = pos;
ParserState state = Delimiter;
delimiters.emplace_back();
for (; *pos; ++pos)
{
switch (state)
{
case Delimiter:
if (*pos == '$')
{
delimiters.back().append(token_begin, pos - token_begin);
++pos;
if (*pos == '{')
{
token_begin = pos + 1;
state = Column;
}
else if (*pos == '$')
{
token_begin = pos;
}
else
throwInvalidFormat("at pos " + std::to_string(pos - format_string.c_str()) +
": expected '{' or '$' after '$'", columnsCount());
}
break;
case Column:
column_names.emplace_back();
pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back());
if (*pos == ':')
state = Format;
else if (*pos == '}')
{
formats.push_back(ColumnFormat::None);
delimiters.emplace_back();
state = Delimiter;
}
else
throwInvalidFormat("Expected ':' or '}' after column name: \"" + column_names.back() + "\"", columnsCount());
token_begin = pos + 1;
format_idx_to_column_idx.emplace_back(idx_by_name(column_names.back()));
break;
case Format:
if (*pos == '}')
{
formats.push_back(stringToFormat(String(token_begin, pos - token_begin)));
token_begin = pos + 1;
delimiters.emplace_back();
state = Delimiter;
}
}
}
if (state != Delimiter)
throwInvalidFormat("Unbalanced parentheses", columnsCount());
delimiters.back().append(token_begin, pos - token_begin);
}
ParsedTemplateFormatString::ColumnFormat ParsedTemplateFormatString::stringToFormat(const String & col_format) const
{
if (col_format.empty())
return ColumnFormat::None;
else if (col_format == "None")
return ColumnFormat::None;
else if (col_format == "Escaped")
return ColumnFormat::Escaped;
else if (col_format == "Quoted")
return ColumnFormat::Quoted;
else if (col_format == "CSV")
return ColumnFormat::Csv;
else if (col_format == "JSON")
return ColumnFormat::Json;
else if (col_format == "XML")
return ColumnFormat::Xml;
else if (col_format == "Raw")
return ColumnFormat::Raw;
else
throwInvalidFormat("Unknown field format " + col_format, columnsCount());
}
size_t ParsedTemplateFormatString::columnsCount() const
{
return format_idx_to_column_idx.size();
}
String ParsedTemplateFormatString::formatToString(ParsedTemplateFormatString::ColumnFormat format)
{
switch (format)
{
case ColumnFormat::None:
return "None";
case ColumnFormat::Escaped:
return "Escaped";
case ColumnFormat::Quoted:
return "Quoted";
case ColumnFormat::Csv:
return "CSV";
case ColumnFormat::Json:
return "Json";
case ColumnFormat::Xml:
return "Xml";
case ColumnFormat::Raw:
return "Raw";
}
__builtin_unreachable();
}
const char * ParsedTemplateFormatString::readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s)
{
s.clear();
if (!size)
return pos;
ReadBufferFromMemory buf{pos, size};
if (*pos == '"')
readDoubleQuotedStringWithSQLStyle(s, buf);
else if (*pos == '`')
readBackQuotedStringWithSQLStyle(s, buf);
else if (isWordCharASCII(*pos))
{
size_t name_size = 1;
while (name_size < size && isWordCharASCII(*(pos + name_size)))
++name_size;
s = String{pos, name_size};
return pos + name_size;
}
return pos + buf.count();
}
String ParsedTemplateFormatString::dump() const
{
WriteBufferFromOwnString res;
res << "Delimiter " << 0 << ": ";
verbosePrintString(delimiters.front().c_str(), delimiters.front().c_str() + delimiters.front().size(), res);
size_t num_columns = std::max(formats.size(), format_idx_to_column_idx.size());
for (size_t i = 0; i < num_columns; ++i)
{
res << "\nColumn " << i << ": \"";
if (column_names.size() <= i)
res << "<ERROR>";
else if (column_names[i].empty())
res << "<SKIPPED>";
else
res << column_names[i];
res << "\" (mapped to table column ";
if (format_idx_to_column_idx.size() <= i)
res << "<ERROR>";
else if (!format_idx_to_column_idx[i])
res << "<SKIPPED>";
else
res << *format_idx_to_column_idx[i];
res << "), Format " << (i < formats.size() ? formatToString(formats[i]) : "<ERROR>");
res << "\nDelimiter " << i + 1 << ": ";
if (delimiters.size() <= i + 1)
res << "<ERROR>";
else
verbosePrintString(delimiters[i + 1].c_str(), delimiters[i + 1].c_str() + delimiters[i + 1].size(), res);
}
return res.str();
}
void ParsedTemplateFormatString::throwInvalidFormat(const String & message, size_t column) const
{
throw Exception("Invalid format string for Template: " + message + " (near column " + std::to_string(column) +
")" + ". Parsed format string:\n" + dump() + "\n",
ErrorCodes::INVALID_TEMPLATE_FORMAT);
}
}

View File

@ -0,0 +1,51 @@
#pragma once
#include <Core/Types.h>
#include <functional>
#include <optional>
namespace DB
{
struct ParsedTemplateFormatString
{
enum class ColumnFormat
{
None,
Escaped,
Quoted,
Csv,
Json,
Xml,
Raw
};
/// Format string has syntax: "Delimiter0 ${ColumnName0:Format0} Delimiter1 ${ColumnName1:Format1} Delimiter2"
/// The following vectors is filled with corresponding values, delimiters.size() - 1 = formats.size() = format_idx_to_column_idx.size()
/// If format_idx_to_column_idx[i] has no value, then TemplateRowInputFormat will skip i-th column.
std::vector<String> delimiters;
std::vector<ColumnFormat> formats;
std::vector<std::optional<size_t>> format_idx_to_column_idx;
/// For diagnostic info
Strings column_names;
typedef std::function<std::optional<size_t>(const String &)> ColumnIdxGetter;
ParsedTemplateFormatString() = default;
ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name);
void parse(const String & format_string, const ColumnIdxGetter & idx_by_name);
ColumnFormat stringToFormat(const String & format) const;
static String formatToString(ColumnFormat format);
static const char * readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s);
size_t columnsCount() const;
String dump() const;
[[noreturn]] void throwInvalidFormat(const String & message, size_t column) const;
};
}

View File

@ -1,504 +0,0 @@
#include <string>
#include <Core/Defines.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Formats/TabSeparatedRowInputStream.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
}
static void skipTSVRow(ReadBuffer & istr, const size_t num_columns)
{
NullSink null_sink;
for (size_t i = 0; i < num_columns; ++i)
{
readEscapedStringInto(null_sink, istr);
assertChar(i == num_columns - 1 ? '\n' : '\t', istr);
}
}
/** Check for a common error case - usage of Windows line feed.
*/
static void checkForCarriageReturn(ReadBuffer & istr)
{
if (istr.position()[0] == '\r' || (istr.position() != istr.buffer().begin() && istr.position()[-1] == '\r'))
throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row."
"\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format."
" You must transform your file to Unix format."
"\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\r.",
ErrorCodes::INCORRECT_DATA);
}
TabSeparatedRowInputStream::TabSeparatedRowInputStream(
ReadBuffer & istr_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: istr(istr_), header(header_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
{
const auto num_columns = header.columns();
data_types.resize(num_columns);
column_indexes_by_names.reserve(num_columns);
for (size_t i = 0; i < num_columns; ++i)
{
const auto & column_info = header.getByPosition(i);
data_types[i] = column_info.type;
column_indexes_by_names.emplace(column_info.name, i);
}
column_indexes_for_input_fields.reserve(num_columns);
read_columns.assign(num_columns, false);
}
void TabSeparatedRowInputStream::setupAllColumnsByTableSchema()
{
read_columns.assign(header.columns(), true);
column_indexes_for_input_fields.resize(header.columns());
for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i)
column_indexes_for_input_fields[i] = i;
}
void TabSeparatedRowInputStream::addInputColumn(const String & column_name)
{
const auto column_it = column_indexes_by_names.find(column_name);
if (column_it == column_indexes_by_names.end())
{
if (format_settings.skip_unknown_fields)
{
column_indexes_for_input_fields.push_back(std::nullopt);
return;
}
throw Exception(
"Unknown field found in TSV header: '" + column_name + "' " +
"at position " + std::to_string(column_indexes_for_input_fields.size()) +
"\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed",
ErrorCodes::INCORRECT_DATA
);
}
const auto column_index = column_it->second;
if (read_columns[column_index])
throw Exception("Duplicate field found while parsing TSV header: " + column_name, ErrorCodes::INCORRECT_DATA);
read_columns[column_index] = true;
column_indexes_for_input_fields.emplace_back(column_index);
}
void TabSeparatedRowInputStream::fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension & row_read_extension)
{
/// It is safe to memorize this on the first run - the format guarantees this does not change
if (unlikely(row_num == 1))
{
columns_to_fill_with_default_values.clear();
for (size_t index = 0; index < read_columns.size(); ++index)
if (read_columns[index] == 0)
columns_to_fill_with_default_values.push_back(index);
}
for (const auto column_index : columns_to_fill_with_default_values)
data_types[column_index]->insertDefaultInto(*columns[column_index]);
row_read_extension.read_columns = read_columns;
}
void TabSeparatedRowInputStream::readPrefix()
{
if (with_names || with_types)
{
/// In this format, we assume that column name or type cannot contain BOM,
/// so, if format has header,
/// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it.
skipBOMIfExists(istr);
}
if (with_names)
{
if (format_settings.with_names_use_header)
{
String column_name;
do
{
readEscapedString(column_name, istr);
addInputColumn(column_name);
}
while (checkChar('\t', istr));
if (!istr.eof())
{
checkForCarriageReturn(istr);
assertChar('\n', istr);
}
}
else
{
setupAllColumnsByTableSchema();
skipTSVRow(istr, column_indexes_for_input_fields.size());
}
}
else
setupAllColumnsByTableSchema();
if (with_types)
{
skipTSVRow(istr, column_indexes_for_input_fields.size());
}
}
bool TabSeparatedRowInputStream::read(MutableColumns & columns, RowReadExtension & ext)
{
if (istr.eof())
return false;
updateDiagnosticInfo();
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
{
const auto & column_index = column_indexes_for_input_fields[input_position];
if (column_index)
{
data_types[*column_index]->deserializeAsTextEscaped(*columns[*column_index], istr, format_settings);
}
else
{
NullSink null_sink;
readEscapedStringInto(null_sink, istr);
}
/// skip separators
if (input_position + 1 < column_indexes_for_input_fields.size())
{
assertChar('\t', istr);
}
else if (!istr.eof())
{
if (unlikely(row_num == 1))
checkForCarriageReturn(istr);
assertChar('\n', istr);
}
}
fillUnreadColumnsWithDefaults(columns, ext);
return true;
}
String TabSeparatedRowInputStream::getDiagnosticInfo()
{
if (istr.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer.
size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current line and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
istr.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
istr.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
/** gcc-7 generates wrong code with optimization level greater than 1.
* See tests: dbms/src/IO/tests/write_int.cpp
* and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh
* This is compiler bug. The bug does not present in gcc-8 and clang-8.
* Nevertheless, we don't need high optimization of this function.
*/
bool OPTIMIZE(1) TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(
MutableColumns & columns, WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
{
if (input_position == 0 && istr.eof())
{
out << "<End of stream>\n";
return false;
}
if (column_indexes_for_input_fields[input_position].has_value())
{
const auto & column_index = *column_indexes_for_input_fields[input_position];
const auto & current_column_type = data_types[column_index];
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(column_index).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(column_index).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
auto prev_position = istr.position();
std::exception_ptr exception;
try
{
current_column_type->deserializeAsTextEscaped(*columns[column_index], istr, format_settings);
}
catch (...)
{
exception = std::current_exception();
}
auto curr_position = istr.position();
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, istr.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue())
{
if (*curr_position != '\n' && *curr_position != '\t')
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, istr.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
NullSink null_sink;
readEscapedStringInto(null_sink, istr);
}
/// Delimiters
if (input_position + 1 == column_indexes_for_input_fields.size())
{
if (!istr.eof())
{
try
{
assertChar('\n', istr);
}
catch (const DB::Exception &)
{
if (*istr.position() == '\t')
{
out << "ERROR: Tab found where line feed is expected."
" It's like your file has more columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped tab in value.\n";
}
else if (*istr.position() == '\r')
{
out << "ERROR: Carriage return found where line feed is expected."
" It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n";
}
else
{
out << "ERROR: There is no line feed. ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
else
{
try
{
assertChar('\t', istr);
}
catch (const DB::Exception &)
{
if (*istr.position() == '\n')
{
out << "ERROR: Line feed found where tab is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n";
}
else if (*istr.position() == '\r')
{
out << "ERROR: Carriage return found where tab is expected.\n";
}
else
{
out << "ERROR: There is no tab. ";
verbosePrintString(istr.position(), istr.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
return true;
}
void TabSeparatedRowInputStream::syncAfterError()
{
skipToUnescapedNextLineOrEOF(istr);
}
void TabSeparatedRowInputStream::updateDiagnosticInfo()
{
++row_num;
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = istr.count() - istr.offset();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = istr.position();
}
void registerInputFormatTabSeparated(FormatFactory & factory)
{
for (auto name : {"TabSeparated", "TSV"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<TabSeparatedRowInputStream>(buf, sample, false, false, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
for (auto name : {"TabSeparatedWithNames", "TSVWithNames"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<TabSeparatedRowInputStream>(buf, sample, true, false, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
for (auto name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"})
{
factory.registerInputFormat(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
UInt64 max_block_size,
UInt64 rows_portion_size,
FormatFactory::ReadCallback callback,
const FormatSettings & settings)
{
return std::make_shared<BlockInputStreamFromRowInputStream>(
std::make_shared<TabSeparatedRowInputStream>(buf, sample, true, true, settings),
sample, max_block_size, rows_portion_size, callback, settings);
});
}
}
}

View File

@ -1,73 +0,0 @@
#pragma once
#include <optional>
#include <unordered_map>
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Formats/IRowInputStream.h>
namespace DB
{
class ReadBuffer;
/** A stream to input data in tsv format.
*/
class TabSeparatedRowInputStream : public IRowInputStream
{
public:
/** with_names - the first line is the header with the names of the columns
* with_types - on the next line header with type names
*/
TabSeparatedRowInputStream(
ReadBuffer & istr_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_);
bool read(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
ReadBuffer & istr;
Block header;
bool with_names;
bool with_types;
const FormatSettings format_settings;
DataTypes data_types;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
std::vector<UInt8> read_columns;
std::vector<size_t> columns_to_fill_with_default_values;
void addInputColumn(const String & column_name);
void setupAllColumnsByTableSchema();
void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension& ext);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
};
}

View File

@ -2,6 +2,3 @@ set(SRCS )
add_executable (tab_separated_streams tab_separated_streams.cpp ${SRCS})
target_link_libraries (tab_separated_streams PRIVATE dbms)
add_executable (block_row_transforms block_row_transforms.cpp ${SRCS})
target_link_libraries (block_row_transforms PRIVATE dbms)

View File

@ -1,57 +0,0 @@
#include <string>
#include <iostream>
#include <fstream>
#include <Core/Block.h>
#include <Core/ColumnWithTypeAndName.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Formats/TabSeparatedRowInputStream.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <DataStreams/copyData.h>
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
#include <Processors/Formats/OutputStreamToOutputFormat.h>
int main(int, char **)
try
{
using namespace DB;
Block sample;
ColumnWithTypeAndName col1;
col1.name = "col1";
col1.type = std::make_shared<DataTypeUInt64>();
col1.column = col1.type->createColumn();
sample.insert(col1);
ColumnWithTypeAndName col2;
col2.name = "col2";
col2.type = std::make_shared<DataTypeString>();
col2.column = col2.type->createColumn();
sample.insert(col2);
ReadBufferFromFile in_buf("test_in");
WriteBufferFromFile out_buf("test_out");
FormatSettings format_settings;
RowInputStreamPtr row_input = std::make_shared<TabSeparatedRowInputStream>(in_buf, sample, false, false, format_settings);
BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings);
BlockOutputStreamPtr block_output = std::make_shared<OutputStreamToOutputFormat>(std::make_shared<TabSeparatedRowOutputFormat>(out_buf, sample, false, false, []{}, format_settings));
copyData(block_input, *block_output);
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
return 1;
}

View File

@ -9,12 +9,12 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <Formats/TabSeparatedRowInputStream.h>
#include <Formats/BlockInputStreamFromRowInputStream.h>
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
#include <DataStreams/copyData.h>
#include <Processors/Formats/OutputStreamToOutputFormat.h>
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
using namespace DB;
@ -39,13 +39,15 @@ try
FormatSettings format_settings;
RowInputStreamPtr row_input = std::make_shared<TabSeparatedRowInputStream>(in_buf, sample, false, false, format_settings);
BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings);
RowInputFormatParams params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, 0, []{}};
InputFormatPtr input_format = std::make_shared<TabSeparatedRowInputFormat>(sample, in_buf, params, false, false, format_settings);
BlockInputStreamPtr block_input = std::make_shared<InputStreamFromInputFormat>(std::move(input_format));
BlockOutputStreamPtr block_output = std::make_shared<OutputStreamToOutputFormat>(
std::make_shared<TabSeparatedRowOutputFormat>(out_buf, sample, false, false, [] {}, format_settings));
copyData(block_input, *block_output);
copyData(*block_input, *block_output);
return 0;
}
catch (...)

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

@ -332,7 +332,7 @@ UInt64 geohashesInBox(const GeohashesInBoxPreparedArgs & args, char * out)
}
}
if (items == 0 && args.items_count != 0)
if (items == 0)
{
size_t l = geohashEncodeImpl(args.longitude_min, args.latitude_min, args.precision, out);
out += l;

View File

@ -336,7 +336,9 @@ void FunctionArrayEnumerateRankedExtended<Derived>::executeMethodImpl(
/// Skipping offsets if no data in this array
if (prev_off == off)
{
want_clear = true;
if (depth_to_look > 2)
want_clear = true;
if (depth_to_look >= 2)
{

View File

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

View File

@ -0,0 +1,316 @@
#include <IO/PeekableReadBuffer.h>
namespace DB
{
PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/,
size_t unread_limit_ /* = default_limit*/)
: BufferWithOwnMemory(start_size_), sub_buf(sub_buf_), unread_limit(unread_limit_)
{
padded &= sub_buf.isPadded();
/// Read from sub-buffer
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf.offset());
checkStateCorrect();
}
bool PeekableReadBuffer::peekNext()
{
checkStateCorrect();
size_t bytes_read = 0;
Position copy_from = pos;
size_t bytes_to_copy = sub_buf.available();
if (useSubbufferOnly())
{
/// Don't have to copy all data from sub-buffer if there is no data in own memory (checkpoint and pos are in sub-buffer)
if (checkpoint)
copy_from = checkpoint;
bytes_read = copy_from - sub_buf.buffer().begin();
bytes_to_copy = sub_buf.buffer().end() - copy_from; /// sub_buf.available();
if (!bytes_to_copy)
{
bytes += bytes_read;
sub_buf.position() = copy_from;
/// Both checkpoint and pos are at the end of sub-buffer. Just load next part of data.
bool res = sub_buf.next();
BufferBase::set(sub_buf.buffer().begin(), sub_buf.buffer().size(), sub_buf.offset());
if (checkpoint)
checkpoint = pos;
checkStateCorrect();
return res;
}
}
/// May throw an exception
resizeOwnMemoryIfNecessary(bytes_to_copy);
if (useSubbufferOnly())
{
bytes += bytes_read;
sub_buf.position() = copy_from;
}
/// Save unread data from sub-buffer to own memory
memcpy(memory.data() + peeked_size, sub_buf.position(), bytes_to_copy);
/// If useSubbufferOnly() is false, then checkpoint is in own memory and it was updated in resizeOwnMemoryIfNecessary
/// Otherwise, checkpoint now at the beginning of own memory
if (checkpoint && useSubbufferOnly())
{
checkpoint = memory.data();
checkpoint_in_own_memory = true;
}
if (currentlyReadFromOwnMemory())
{
/// Update buffer size
BufferBase::set(memory.data(), peeked_size + bytes_to_copy, offset());
}
else
{
/// Switch to reading from own memory
size_t pos_offset = peeked_size + this->offset();
if (useSubbufferOnly())
{
if (checkpoint)
pos_offset = bytes_to_copy;
else
pos_offset = 0;
}
BufferBase::set(memory.data(), peeked_size + bytes_to_copy, pos_offset);
}
peeked_size += bytes_to_copy;
sub_buf.position() += bytes_to_copy;
checkStateCorrect();
return sub_buf.next();
}
void PeekableReadBuffer::setCheckpoint()
{
checkStateCorrect();
#ifndef NDEBUG
if (checkpoint)
throw DB::Exception("Does not support recursive checkpoints.", ErrorCodes::LOGICAL_ERROR);
#endif
checkpoint_in_own_memory = currentlyReadFromOwnMemory();
if (!checkpoint_in_own_memory)
{
/// Don't need to store unread data anymore
peeked_size = 0;
}
checkpoint = pos;
checkStateCorrect();
}
void PeekableReadBuffer::dropCheckpoint()
{
checkStateCorrect();
#ifndef NDEBUG
if (!checkpoint)
throw DB::Exception("There is no checkpoint", ErrorCodes::LOGICAL_ERROR);
#endif
if (!currentlyReadFromOwnMemory())
{
/// Don't need to store unread data anymore
peeked_size = 0;
}
checkpoint = nullptr;
checkpoint_in_own_memory = false;
checkStateCorrect();
}
void PeekableReadBuffer::rollbackToCheckpoint()
{
checkStateCorrect();
if (!checkpoint)
throw DB::Exception("There is no checkpoint", ErrorCodes::LOGICAL_ERROR);
else if (checkpointInOwnMemory() == currentlyReadFromOwnMemory())
pos = checkpoint;
else /// Checkpoint is in own memory and pos is not. Switch to reading from own memory
BufferBase::set(memory.data(), peeked_size, checkpoint - memory.data());
checkStateCorrect();
}
bool PeekableReadBuffer::nextImpl()
{
/// FIXME wrong bytes count because it can read the same data again after rollbackToCheckpoint()
/// However, changing bytes count on every call of next() (even after rollback) allows to determine if some pointers were invalidated.
checkStateCorrect();
bool res;
if (!checkpoint)
{
if (!useSubbufferOnly())
{
/// All copied data have been read from own memory, continue reading from sub_buf
peeked_size = 0;
res = sub_buf.hasPendingData() || sub_buf.next();
}
else
{
/// Load next data to sub_buf
sub_buf.position() = pos;
res = sub_buf.next();
}
Buffer & sub_working = sub_buf.buffer();
/// Switch to reading from sub_buf (or just update it if already switched)
BufferBase::set(sub_working.begin(), sub_working.size(), 0);
}
else
{
if (currentlyReadFromOwnMemory())
res = sub_buf.hasPendingData() || sub_buf.next();
else
res = peekNext();
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), 0);
}
checkStateCorrect();
return res;
}
bool PeekableReadBuffer::useSubbufferOnly() const
{
return !peeked_size;
}
void PeekableReadBuffer::checkStateCorrect() const
{
#ifndef NDEBUG
if (checkpoint)
{
if (checkpointInOwnMemory())
{
if (!peeked_size)
throw DB::Exception("Checkpoint in empty own buffer", ErrorCodes::LOGICAL_ERROR);
if (currentlyReadFromOwnMemory() && pos < checkpoint)
throw DB::Exception("Current position in own buffer before checkpoint in own buffer", ErrorCodes::LOGICAL_ERROR);
}
else
{
if (peeked_size)
throw DB::Exception("Own buffer is not empty", ErrorCodes::LOGICAL_ERROR);
if (currentlyReadFromOwnMemory())
throw DB::Exception("Current position in own buffer before checkpoint in subbuffer", ErrorCodes::LOGICAL_ERROR);
if (pos < checkpoint)
throw DB::Exception("Current position in subbuffer before checkpoint in subbuffer", ErrorCodes::LOGICAL_ERROR);
}
}
else
{
if (!currentlyReadFromOwnMemory() && peeked_size)
throw DB::Exception("Own buffer is not empty", ErrorCodes::LOGICAL_ERROR);
}
if (currentlyReadFromOwnMemory() && !peeked_size)
throw DB::Exception("Pos in empty own buffer", ErrorCodes::LOGICAL_ERROR);
if (unread_limit < memory.size())
throw DB::Exception("Size limit exceed", ErrorCodes::LOGICAL_ERROR);
#endif
}
size_t PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append)
{
checkStateCorrect();
bool needUpdateCheckpoint = checkpointInOwnMemory();
bool needUpdatePos = currentlyReadFromOwnMemory();
size_t offset = 0;
if (needUpdateCheckpoint)
offset = checkpoint - memory.data();
else if (needUpdatePos)
offset = this->offset();
size_t new_size = peeked_size + bytes_to_append;
if (memory.size() < new_size)
{
if (bytes_to_append < offset && 2 * (peeked_size - offset) <= memory.size())
{
/// Move unread data to the beginning of own memory instead of resize own memory
peeked_size -= offset;
memmove(memory.data(), memory.data() + offset, peeked_size);
bytes += offset;
if (needUpdateCheckpoint)
checkpoint -= offset;
if (needUpdatePos)
pos -= offset;
checkStateCorrect();
return 0;
}
else
{
if (unread_limit < new_size)
throw DB::Exception("PeekableReadBuffer: Memory limit exceed", ErrorCodes::MEMORY_LIMIT_EXCEEDED);
size_t pos_offset = pos - memory.data();
size_t new_size_amortized = memory.size() * 2;
if (new_size_amortized < new_size)
new_size_amortized = new_size;
else if (unread_limit < new_size_amortized)
new_size_amortized = unread_limit;
memory.resize(new_size_amortized);
if (needUpdateCheckpoint)
checkpoint = memory.data() + offset;
if (needUpdatePos)
{
BufferBase::set(memory.data(), peeked_size, pos_offset);
}
}
}
checkStateCorrect();
return offset;
}
PeekableReadBuffer::~PeekableReadBuffer()
{
if (!currentlyReadFromOwnMemory())
sub_buf.position() = pos;
}
std::shared_ptr<BufferWithOwnMemory<ReadBuffer>> PeekableReadBuffer::takeUnreadData()
{
checkStateCorrect();
if (!currentlyReadFromOwnMemory())
return std::make_shared<BufferWithOwnMemory<ReadBuffer>>(0);
size_t unread_size = memory.data() + peeked_size - pos;
auto unread = std::make_shared<BufferWithOwnMemory<ReadBuffer>>(unread_size);
memcpy(unread->buffer().begin(), pos, unread_size);
unread->BufferBase::set(unread->buffer().begin(), unread_size, 0);
peeked_size = 0;
checkpoint = nullptr;
checkpoint_in_own_memory = false;
BufferBase::set(sub_buf.buffer().begin(), sub_buf.buffer().size(), sub_buf.offset());
checkStateCorrect();
return unread;
}
bool PeekableReadBuffer::currentlyReadFromOwnMemory() const
{
return working_buffer.begin() != sub_buf.buffer().begin();
}
bool PeekableReadBuffer::checkpointInOwnMemory() const
{
return checkpoint_in_own_memory;
}
void PeekableReadBuffer::assertCanBeDestructed() const
{
if (peeked_size && pos != memory.data() + peeked_size)
throw DB::Exception("There are data, which were extracted from sub-buffer, but not from peekable buffer. "
"Cannot destruct peekable buffer correctly because tha data will be lost."
"Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR);
}
}

View File

@ -0,0 +1,96 @@
#pragma once
#include <IO/ReadBuffer.h>
#include <IO/BufferWithOwnMemory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int MEMORY_LIMIT_EXCEEDED;
extern const int LOGICAL_ERROR;
}
/// Allows to peek next part of data from sub-buffer without extracting it.
/// Also allows to set checkpoint at some position in stream and come back to this position later,
/// even if next() was called.
/// Sub-buffer should not be accessed directly during the lifelime of peekable buffer.
/// If position() of peekable buffer is explicitly set to some position before checkpoint
/// (e.g. by istr.position() = prev_pos), behavior is undefined.
class PeekableReadBuffer : public BufferWithOwnMemory<ReadBuffer>
{
friend class PeekableReadBufferCheckpoint;
public:
explicit PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE,
size_t unread_limit_ = 16 * DBMS_DEFAULT_BUFFER_SIZE);
/// Use takeUnreadData() to extract unread data before destruct object
~PeekableReadBuffer() override;
/// Saves unread data to own memory, so it will be possible to read it later. Loads next data to sub-buffer.
/// Doesn't change checkpoint and position in stream,
/// but all pointers (such as this->buffer().end() and this->position()) may be invalidated
/// @returns false in case of EOF in sub-buffer, otherwise returns true
bool peekNext();
Buffer & lastPeeked() { return sub_buf.buffer(); }
/// Sets checkpoint at current position
void setCheckpoint();
/// Forget checkpoint and all data between checkpoint and position
void dropCheckpoint();
/// Sets position at checkpoint.
/// All pointers (such as this->buffer().end()) may be invalidated
void rollbackToCheckpoint();
/// If position is in own memory, returns buffer with data, which were extracted from sub-buffer,
/// but not from this buffer, so the data will not be lost after destruction of this buffer.
/// If position is in sub-buffer, returns empty buffer.
std::shared_ptr<BufferWithOwnMemory<ReadBuffer>> takeUnreadData();
void assertCanBeDestructed() const;
private:
bool nextImpl() override;
inline bool useSubbufferOnly() const;
inline bool currentlyReadFromOwnMemory() const;
inline bool checkpointInOwnMemory() const;
void checkStateCorrect() const;
/// Makes possible to append `bytes_to_append` bytes to data in own memory.
/// Updates all invalidated pointers and sizes.
/// @returns new offset of unread data in own memory
size_t resizeOwnMemoryIfNecessary(size_t bytes_to_append);
ReadBuffer & sub_buf;
const size_t unread_limit;
size_t peeked_size = 0;
Position checkpoint = nullptr;
bool checkpoint_in_own_memory = false;
};
class PeekableReadBufferCheckpoint : boost::noncopyable
{
PeekableReadBuffer & buf;
bool auto_rollback;
public:
explicit PeekableReadBufferCheckpoint(PeekableReadBuffer & buf_, bool auto_rollback_ = false)
: buf(buf_), auto_rollback(auto_rollback_) { buf.setCheckpoint(); }
~PeekableReadBufferCheckpoint()
{
if (!buf.checkpoint)
return;
if (auto_rollback)
buf.rollbackToCheckpoint();
buf.dropCheckpoint();
}
};
}

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

@ -0,0 +1,131 @@
#include <gtest/gtest.h>
#include <Core/Types.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromString.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/PeekableReadBuffer.h>
void readAndAssert(DB::ReadBuffer & buf, const char * str)
{
size_t n = strlen(str);
char tmp[n];
buf.readStrict(tmp, n);
ASSERT_EQ(strncmp(tmp, str, n), 0);
}
void assertAvailable(DB::ReadBuffer & buf, const char * str)
{
size_t n = strlen(str);
ASSERT_EQ(buf.available(), n);
ASSERT_EQ(strncmp(buf.position(), str, n), 0);
}
TEST(PeekableReadBuffer, CheckpointsWorkCorrectly)
try
{
std::string s1 = "0123456789";
std::string s2 = "qwertyuiop";
std::string s3 = "asdfghjkl;";
std::string s4 = "zxcvbnm,./";
DB::ReadBufferFromString b1(s1);
DB::ReadBufferFromString b2(s2);
DB::ReadBufferFromString b3(s3);
DB::ReadBufferFromString b4(s4);
DB::ConcatReadBuffer concat({&b1, &b2, &b3, &b4});
DB::PeekableReadBuffer peekable(concat, 0, 16);
ASSERT_TRUE(!peekable.eof());
assertAvailable(peekable, "0123456789");
{
DB::PeekableReadBufferCheckpoint checkpoint{peekable};
readAndAssert(peekable, "01234");
}
bool exception = false;
try
{
peekable.rollbackToCheckpoint();
}
catch (DB::Exception & e)
{
if (e.code() != DB::ErrorCodes::LOGICAL_ERROR)
throw;
exception = true;
}
ASSERT_TRUE(exception);
assertAvailable(peekable, "56789");
readAndAssert(peekable, "56");
peekable.setCheckpoint();
readAndAssert(peekable, "789qwertyu");
peekable.rollbackToCheckpoint();
peekable.dropCheckpoint();
assertAvailable(peekable, "789");
peekable.peekNext();
assertAvailable(peekable, "789qwertyuiop");
ASSERT_EQ(peekable.lastPeeked().size(), 10);
ASSERT_EQ(strncmp(peekable.lastPeeked().begin(), "asdfghjkl;", 10), 0);
exception = false;
try
{
DB::PeekableReadBufferCheckpoint checkpoint{peekable, true};
peekable.ignore(30);
}
catch (DB::Exception & e)
{
if (e.code() != DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED)
throw;
exception = true;
}
ASSERT_TRUE(exception);
assertAvailable(peekable, "789qwertyuiop");
ASSERT_EQ(peekable.lastPeeked().size(), 10);
ASSERT_EQ(strncmp(peekable.lastPeeked().begin(), "asdfghjkl;", 10), 0);
readAndAssert(peekable, "789qwertyu");
peekable.setCheckpoint();
readAndAssert(peekable, "iopasdfghj");
assertAvailable(peekable, "kl;");
peekable.dropCheckpoint();
peekable.setCheckpoint();
readAndAssert(peekable, "kl;zxcvbnm,./");
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
peekable.rollbackToCheckpoint();
readAndAssert(peekable, "kl;zxcvbnm");
peekable.dropCheckpoint();
exception = false;
try
{
peekable.assertCanBeDestructed();
}
catch (DB::Exception & e)
{
if (e.code() != DB::ErrorCodes::LOGICAL_ERROR)
throw;
exception = true;
}
ASSERT_TRUE(exception);
auto buf_ptr = peekable.takeUnreadData();
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
ASSERT_TRUE(peekable.eof());
readAndAssert(*buf_ptr, ",./");
ASSERT_TRUE(buf_ptr->eof());
peekable.assertCanBeDestructed();
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -745,6 +745,30 @@ Block Aggregator::convertOneBucketToBlock(
return block;
}
Block Aggregator::mergeAndConvertOneBucketToBlock(
ManyAggregatedDataVariants & variants,
Arena * arena,
bool final,
size_t bucket) const
{
auto & merged_data = *variants[0];
auto method = merged_data.type;
Block block;
if (false) {}
#define M(NAME) \
else if (method == AggregatedDataVariants::Type::NAME) \
{ \
mergeBucketImpl<decltype(merged_data.NAME)::element_type>(variants, bucket, arena); \
block = convertOneBucketToBlock(merged_data, *merged_data.NAME, final, bucket); \
}
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
#undef M
return block;
}
template <typename Method>
void Aggregator::writeToTemporaryFileImpl(
@ -1654,9 +1678,7 @@ private:
}
};
std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
ManyAggregatedDataVariants & data_variants, bool final, size_t max_threads) const
ManyAggregatedDataVariants Aggregator::prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants) const
{
if (data_variants.empty())
throw Exception("Empty data passed to Aggregator::mergeAndConvertToBlocks.", ErrorCodes::EMPTY_DATA_PASSED);
@ -1670,7 +1692,7 @@ std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
non_empty_data.push_back(data);
if (non_empty_data.empty())
return std::make_unique<NullBlockInputStream>(getHeader(final));
return {};
if (non_empty_data.size() > 1)
{
@ -1714,6 +1736,17 @@ std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
non_empty_data[i]->aggregates_pools.begin(), non_empty_data[i]->aggregates_pools.end());
}
return non_empty_data;
}
std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
ManyAggregatedDataVariants & data_variants, bool final, size_t max_threads) const
{
ManyAggregatedDataVariants non_empty_data = prepareVariantsToMerge(data_variants);
if (non_empty_data.empty())
return std::make_unique<NullBlockInputStream>(getHeader(final));
return std::make_unique<MergingAndConvertingBlockInputStream>(*this, non_empty_data, final, max_threads);
}

View File

@ -733,6 +733,7 @@ struct AggregatedDataVariants : private boost::noncopyable
using AggregatedDataVariantsPtr = std::shared_ptr<AggregatedDataVariants>;
using ManyAggregatedDataVariants = std::vector<AggregatedDataVariantsPtr>;
using ManyAggregatedDataVariantsPtr = std::shared_ptr<ManyAggregatedDataVariants>;
/** How are "total" values calculated with WITH TOTALS?
* (For more details, see TotalsHavingBlockInputStream.)
@ -850,6 +851,7 @@ public:
/** Merge several aggregation data structures and output the result as a block stream.
*/
std::unique_ptr<IBlockInputStream> mergeAndConvertToBlocks(ManyAggregatedDataVariants & data_variants, bool final, size_t max_threads) const;
ManyAggregatedDataVariants prepareVariantsToMerge(ManyAggregatedDataVariants & data_variants) const;
/** Merge the stream of partially aggregated blocks into one data structure.
* (Pre-aggregate several blocks that represent the result of independent aggregations from remote servers.)
@ -904,6 +906,8 @@ public:
protected:
friend struct AggregatedDataVariants;
friend class MergingAndConvertingBlockInputStream;
friend class ConvertingAggregatedToChunksTransform;
friend class ConvertingAggregatedToChunksSource;
Params params;
@ -1084,6 +1088,12 @@ protected:
bool final,
size_t bucket) const;
Block mergeAndConvertOneBucketToBlock(
ManyAggregatedDataVariants & variants,
Arena * arena,
bool final,
size_t bucket) const;
Block prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const;
Block prepareBlockAndFillSingleLevel(AggregatedDataVariants & data_variants, bool final) const;
BlocksList prepareBlocksAndFillTwoLevel(AggregatedDataVariants & data_variants, bool final, ThreadPool * thread_pool) const;
@ -1155,5 +1165,4 @@ APPLY_FOR_AGGREGATED_VARIANTS(M)
#undef M
}

View File

@ -1,13 +1,18 @@
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <Interpreters/Join.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/ASTSelectQuery.h>
#include <Core/Block.h>
#include <Storages/IStorage.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
{
@ -26,7 +31,6 @@ void AnalyzedJoin::addUsingKey(const ASTPtr & ast)
void AnalyzedJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
{
with_using = false;
key_names_left.push_back(left_table_ast->getColumnName());
key_names_right.push_back(right_table_ast->getAliasOrColumnName());
@ -37,7 +41,7 @@ void AnalyzedJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast)
/// @return how many times right key appears in ON section.
size_t AnalyzedJoin::rightKeyInclusion(const String & name) const
{
if (with_using)
if (hasUsing())
return 0;
size_t count = 0;
@ -101,6 +105,143 @@ std::unordered_map<String, String> AnalyzedJoin::getOriginalColumnsMap(const Nam
return out;
}
ASTPtr AnalyzedJoin::leftKeysList() const
{
ASTPtr keys_list = std::make_shared<ASTExpressionList>();
keys_list->children = key_asts_left;
return keys_list;
}
ASTPtr AnalyzedJoin::rightKeysList() const
{
ASTPtr keys_list = std::make_shared<ASTExpressionList>();
if (hasOn())
keys_list->children = key_asts_right;
return keys_list;
}
Names AnalyzedJoin::requiredJoinedNames() const
{
NameSet required_columns_set(key_names_right.begin(), key_names_right.end());
for (const auto & joined_column : columns_added_by_join)
required_columns_set.insert(joined_column.name);
return Names(required_columns_set.begin(), required_columns_set.end());
}
void AnalyzedJoin::appendRequiredColumns(const Block & sample, NameSet & required_columns) const
{
for (auto & column : key_names_right)
if (!sample.has(column))
required_columns.insert(column);
for (auto & column : columns_added_by_join)
if (!sample.has(column.name))
required_columns.insert(column.name);
}
void AnalyzedJoin::addJoinedColumn(const NameAndTypePair & joined_column)
{
if (join_use_nulls && isLeftOrFull(table_join.kind))
{
auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type;
columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type)));
}
else
columns_added_by_join.push_back(joined_column);
}
void AnalyzedJoin::addJoinedColumnsAndCorrectNullability(Block & sample_block) const
{
bool right_or_full_join = isRightOrFull(table_join.kind);
bool left_or_full_join = isLeftOrFull(table_join.kind);
for (auto & col : sample_block)
{
/// Materialize column.
/// Column is not empty if it is constant, but after Join all constants will be materialized.
/// So, we need remove constants from header.
if (col.column)
col.column = nullptr;
bool make_nullable = join_use_nulls && right_or_full_join;
if (make_nullable && col.type->canBeInsideNullable())
col.type = makeNullable(col.type);
}
for (const auto & col : columns_added_by_join)
{
auto res_type = col.type;
bool make_nullable = join_use_nulls && left_or_full_join;
if (!make_nullable)
{
/// Keys from right table are usually not stored in Join, but copied from the left one.
/// So, if left key is nullable, let's make right key nullable too.
/// Note: for some join types it's not needed and, probably, may be removed.
/// Note: changing this code, take into account the implementation in Join.cpp.
auto it = std::find(key_names_right.begin(), key_names_right.end(), col.name);
if (it != key_names_right.end())
{
auto pos = it - key_names_right.begin();
const auto & left_key_name = key_names_left[pos];
make_nullable = sample_block.getByName(left_key_name).type->isNullable();
}
}
if (make_nullable && res_type->canBeInsideNullable())
res_type = makeNullable(res_type);
sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name));
}
}
bool AnalyzedJoin::sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y)
{
if (!x && !y)
return true;
if (!x || !y)
return false;
return x->table_join.kind == y->table_join.kind
&& 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->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
{
auto join = std::make_shared<Join>(key_names_right, join_use_nulls, size_limits_for_join, table_join.kind, table_join.strictness);
join->setSampleBlock(sample_block);
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

@ -2,7 +2,9 @@
#include <Core/Names.h>
#include <Core/NamesAndTypes.h>
#include <Parsers/IAST.h>
#include <Core/SettingsCommon.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <utility>
#include <memory>
@ -13,8 +15,12 @@ namespace DB
class Context;
class ASTSelectQuery;
struct DatabaseAndTableWithAlias;
class Block;
struct AnalyzedJoin
class Join;
using JoinPtr = std::shared_ptr<Join>;
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.
@ -28,31 +34,33 @@ struct AnalyzedJoin
* It's possible to use name `expr(t2 columns)`.
*/
private:
friend class SyntaxAnalyzer;
friend struct SyntaxAnalyzerResult;
friend class ExpressionAnalyzer;
friend class SelectQueryExpressionAnalyzer;
Names key_names_left;
Names key_names_right; /// Duplicating names are qualified.
ASTs key_asts_left;
ASTs key_asts_right;
bool with_using = true;
ASTTableJoin table_join;
bool join_use_nulls = false;
/// All columns which can be read from joined table. Duplicating names are qualified.
NamesAndTypesList columns_from_joined_table;
/// Columns will be added to block by JOIN. It's a subset of columns_from_joined_table with corrected Nullability
NamesAndTypesList columns_added_by_join;
/// Name -> original name. Names are the same as in columns_from_joined_table list.
std::unordered_map<String, String> original_names;
/// 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);
bool hasUsing() const { return with_using; }
bool hasOn() const { return !with_using; }
bool hasUsing() const { return table_join.using_expression_list != nullptr; }
bool hasOn() const { return !hasUsing(); }
NameSet getQualifiedColumnsSet() const;
NameSet getOriginalColumnsSet() const;
@ -60,6 +68,27 @@ public:
void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix);
size_t rightKeyInclusion(const String & name) const;
void appendRequiredColumns(const Block & sample, NameSet & required_columns) const;
void addJoinedColumn(const NameAndTypePair & joined_column);
void addJoinedColumnsAndCorrectNullability(Block & sample_block) const;
ASTPtr leftKeysList() const;
ASTPtr rightKeysList() const; /// For ON syntax only
Names requiredJoinedNames() const;
const Names & keyNamesLeft() const { return key_names_left; }
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);
};
struct ASTTableExpression;

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,16 +3,16 @@
#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>
#include <optional>
#include <DataTypes/DataTypeNullable.h>
namespace ProfileEvents
@ -45,7 +45,8 @@ Names ExpressionAction::getNeededColumns() const
res.insert(res.end(), array_joined_columns.begin(), array_joined_columns.end());
res.insert(res.end(), join_key_names_left.begin(), join_key_names_left.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,20 +160,11 @@ ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_column
return a;
}
ExpressionAction ExpressionAction::ordinaryJoin(
const ASTTableJoin & join_params,
std::shared_ptr<const Join> join_,
const Names & join_key_names_left,
const Names & join_key_names_right,
const NamesAndTypesList & columns_added_by_join_)
ExpressionAction ExpressionAction::ordinaryJoin(std::shared_ptr<AnalyzedJoin> table_join)
{
ExpressionAction a;
a.type = JOIN;
a.join = std::move(join_);
a.join_kind = join_params.kind;
a.join_key_names_left = join_key_names_left;
a.join_key_names_right = join_key_names_right;
a.columns_added_by_join = columns_added_by_join_;
a.table_join = table_join;
return a;
}
@ -277,51 +269,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings,
case JOIN:
{
bool is_null_used_as_default = settings.join_use_nulls;
bool right_or_full_join = isRightOrFull(join_kind);
bool left_or_full_join = isLeftOrFull(join_kind);
for (auto & col : sample_block)
{
/// Materialize column.
/// Column is not empty if it is constant, but after Join all constants will be materialized.
/// So, we need remove constants from header.
if (col.column)
col.column = nullptr;
bool make_nullable = is_null_used_as_default && right_or_full_join;
if (make_nullable && col.type->canBeInsideNullable())
col.type = makeNullable(col.type);
}
for (const auto & col : columns_added_by_join)
{
auto res_type = col.type;
bool make_nullable = is_null_used_as_default && left_or_full_join;
if (!make_nullable)
{
/// Keys from right table are usually not stored in Join, but copied from the left one.
/// So, if left key is nullable, let's make right key nullable too.
/// Note: for some join types it's not needed and, probably, may be removed.
/// Note: changing this code, take into account the implementation in Join.cpp.
auto it = std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name);
if (it != join_key_names_right.end())
{
auto pos = it - join_key_names_right.begin();
const auto & left_key_name = join_key_names_left[pos];
make_nullable = sample_block.getByName(left_key_name).type->isNullable();
}
}
if (make_nullable && res_type->canBeInsideNullable())
res_type = makeNullable(res_type);
sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name));
}
table_join->addJoinedColumnsAndCorrectNullability(sample_block);
break;
}
@ -527,7 +475,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const
case JOIN:
{
join->joinBlock(block, join_key_names_left, columns_added_by_join);
table_join->joinBlock(block);
break;
}
@ -595,7 +543,7 @@ void ExpressionAction::executeOnTotals(Block & block) const
if (type != JOIN)
execute(block, false);
else
join->joinTotals(block);
table_join->joinTotals(block);
}
@ -645,9 +593,10 @@ std::string ExpressionAction::toString() const
case JOIN:
ss << "JOIN ";
for (NamesAndTypesList::const_iterator it = columns_added_by_join.begin(); it != columns_added_by_join.end(); ++it)
for (NamesAndTypesList::const_iterator it = table_join->columnsAddedByJoin().begin();
it != table_join->columnsAddedByJoin().end(); ++it)
{
if (it != columns_added_by_join.begin())
if (it != table_join->columnsAddedByJoin().begin())
ss << ", ";
ss << it->name;
}
@ -813,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
@ -1215,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_key_names_left, action.columns_added_by_join, max_block_size);
if (action.table_join)
return action.table_join;
return {};
}
@ -1267,7 +1207,7 @@ UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action
hash.update(col);
break;
case JOIN:
for (const auto & col : action.columns_added_by_join)
for (const auto & col : action.table_join->columnsAddedByJoin())
hash.update(col.name);
break;
case PROJECT:
@ -1325,10 +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
&& join_key_names_left == other.join_key_names_left
&& join_key_names_right == other.join_key_names_right
&& columns_added_by_join == other.columns_added_by_join
&& AnalyzedJoin::sameJoin(table_join.get(), other.table_join.get())
&& projection == other.projection
&& is_function_compiled == other.is_function_compiled;
}

View File

@ -4,7 +4,6 @@
#include <Core/ColumnWithTypeAndName.h>
#include <Core/Names.h>
#include <Core/Settings.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <Interpreters/Context.h>
#include <Common/SipHash.h>
#include "config_core.h"
@ -24,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>;
@ -104,11 +103,7 @@ public:
bool unaligned_array_join = false;
/// For JOIN
std::shared_ptr<const Join> join;
ASTTableJoin::Kind join_kind;
Names join_key_names_left;
Names join_key_names_right;
NamesAndTypesList columns_added_by_join;
std::shared_ptr<const AnalyzedJoin> table_join;
/// For PROJECT.
NamesWithAliases projection;
@ -124,9 +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(const ASTTableJoin & join_params, std::shared_ptr<const Join> join_,
const Names & join_key_names_left, const Names & join_key_names_right,
const NamesAndTypesList & columns_added_by_join_);
static ExpressionAction ordinaryJoin(std::shared_ptr<AnalyzedJoin> join);
/// Which columns necessary to perform this action.
Names getNeededColumns() const;
@ -242,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

@ -29,7 +29,6 @@
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/Set.h>
#include <Interpreters/Join.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
@ -134,14 +133,8 @@ void ExpressionAnalyzer::analyzeAggregation()
const ASTTablesInSelectQueryElement * join = select_query->join();
if (join)
{
const auto & table_join = join->table_join->as<ASTTableJoin &>();
if (table_join.using_expression_list)
getRootActions(table_join.using_expression_list, true, temp_actions);
if (table_join.on_expression)
for (const auto & key_ast : analyzedJoin().key_asts_left)
getRootActions(key_ast, true, temp_actions);
addJoinAction(table_join, temp_actions);
getRootActions(analyzedJoin().leftKeysList(), true, temp_actions);
addJoinAction(temp_actions);
}
}
@ -298,7 +291,8 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node)
{
NamesAndTypesList temp_columns = sourceColumns();
temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end());
temp_columns.insert(temp_columns.end(), columnsAddedByJoin().begin(), columnsAddedByJoin().end());
temp_columns.insert(temp_columns.end(),
analyzedJoin().columnsAddedByJoin().begin(), analyzedJoin().columnsAddedByJoin().end());
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(temp_columns, context);
getRootActions(left_in_operand, true, temp_actions);
@ -412,22 +406,9 @@ bool SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & cha
return true;
}
static void appendRequiredColumns(
NameSet & required_columns, const Block & sample, const Names & key_names_right, const NamesAndTypesList & columns_added_by_join)
void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions) const
{
for (auto & column : key_names_right)
if (!sample.has(column))
required_columns.insert(column);
for (auto & column : columns_added_by_join)
if (!sample.has(column.name))
required_columns.insert(column.name);
}
/// It's possible to set nullptr as join for only_types mode
void ExpressionAnalyzer::addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, JoinPtr join) const
{
actions->add(ExpressionAction::ordinaryJoin(join_params, std::move(join), analyzedJoin().key_names_left, analyzedJoin().key_names_right, columnsAddedByJoin()));
actions->add(ExpressionAction::ordinaryJoin(syntax->analyzed_join));
}
bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types)
@ -437,18 +418,13 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b
return false;
SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join);
ASTPtr left_keys_list = std::make_shared<ASTExpressionList>();
left_keys_list->children = analyzedJoin().key_asts_left;
syntax->analyzed_join->setHashJoin(subquery_for_set.join);
initChain(chain, sourceColumns());
ExpressionActionsChain::Step & step = chain.steps.back();
auto & join_params = ast_join->table_join->as<ASTTableJoin &>();
getRootActions(left_keys_list, only_types, step.actions);
addJoinAction(join_params, step.actions, subquery_for_set.join);
getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions);
addJoinAction(step.actions);
return true;
}
@ -524,11 +500,9 @@ void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryEle
Names action_columns = joined_block_actions->getRequiredColumns();
NameSet required_columns(action_columns.begin(), action_columns.end());
auto & analyzed_join = analyzedJoin();
appendRequiredColumns(
required_columns, joined_block_actions->getSampleBlock(), analyzed_join.key_names_right, columnsAddedByJoin());
analyzedJoin().appendRequiredColumns(joined_block_actions->getSampleBlock(), required_columns);
auto original_map = analyzed_join.getOriginalColumnsMap(required_columns);
auto original_map = analyzedJoin().getOriginalColumnsMap(required_columns);
Names original_columns;
for (auto & pr : original_map)
original_columns.push_back(pr.second);
@ -542,29 +516,16 @@ void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryEle
joined_block_actions->execute(sample_block);
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
auto & join_params = join_element.table_join->as<ASTTableJoin &>();
subquery_for_set.join = std::make_shared<Join>(analyzedJoin().key_names_right, settings.join_use_nulls,
settings.size_limits_for_join, join_params.kind, join_params.strictness);
subquery_for_set.join->setSampleBlock(sample_block);
subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join);
subquery_for_set.joined_block_actions = joined_block_actions;
}
ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const
{
/// Create custom expression list with join keys from right table.
ASTPtr expression_list = std::make_shared<ASTExpressionList>();
ASTs & children = expression_list->children;
ASTPtr expression_list = analyzedJoin().rightKeysList();
Names required_columns = analyzedJoin().requiredJoinedNames();
if (analyzedJoin().hasOn())
for (const auto & join_right_key : analyzedJoin().key_asts_right)
children.emplace_back(join_right_key);
NameSet required_columns_set(analyzedJoin().key_names_right.begin(), analyzedJoin().key_names_right.end());
for (const auto & joined_column : columnsAddedByJoin())
required_columns_set.insert(joined_column.name);
Names required_columns(required_columns_set.begin(), required_columns_set.end());
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columns_from_joined_table, required_columns);
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columnsFromJoinedTable(), required_columns);
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
}

View File

@ -121,9 +121,8 @@ protected:
SyntaxAnalyzerResultPtr syntax;
const StoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists.
const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; }
const AnalyzedJoin & analyzedJoin() const { return *syntax->analyzed_join; }
const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; }
const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; }
const std::vector<const ASTFunction *> & aggregates() const { return syntax->aggregates; }
/// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables.
@ -131,7 +130,7 @@ protected:
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const;
void addJoinAction(const ASTTableJoin & join_params, 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>
@ -1038,20 +1039,18 @@ 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())
{
auto stream = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(
header_before_join, settings.max_block_size);
if constexpr (pipeline_with_processors)
if (auto stream = join->createStreamWithNonJoinedDataIfFullOrRightJoin(header_before_join, settings.max_block_size))
{
auto source = std::make_shared<SourceFromInputStream>(std::move(stream));
pipeline.addDelayedStream(source);
if constexpr (pipeline_with_processors)
{
auto source = std::make_shared<SourceFromInputStream>(std::move(stream));
pipeline.addDelayedStream(source);
}
else
pipeline.stream_with_non_joined_data = std::move(stream);
}
else
pipeline.stream_with_non_joined_data = std::move(stream);
}
}
@ -2081,10 +2080,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);
@ -2096,6 +2093,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

@ -10,6 +10,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Join.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/joinDispatch.h>
#include <Interpreters/NullableUtils.h>
@ -1048,8 +1049,11 @@ void Join::joinGet(Block & block, const String & column_name) const
}
void Join::joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const
void Join::joinBlock(Block & block, const AnalyzedJoin & join_params) const
{
const Names & key_names_left = join_params.keyNamesLeft();
const NamesAndTypesList & columns_added_by_join = join_params.columnsAddedByJoin();
std::shared_lock lock(rwlock);
checkTypesOfKeys(block, key_names_left, sample_block_with_keys);
@ -1457,10 +1461,11 @@ private:
};
BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, const Names & key_names_left,
const NamesAndTypesList & columns_added_by_join, UInt64 max_block_size) const
BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, const AnalyzedJoin & join_params,
UInt64 max_block_size) const
{
return std::make_shared<NonJoinedBlockInputStream>(*this, left_sample_block, key_names_left, columns_added_by_join, max_block_size);
return std::make_shared<NonJoinedBlockInputStream>(*this, left_sample_block,
join_params.keyNamesLeft(), join_params.columnsAddedByJoin(), max_block_size);
}

View File

@ -26,6 +26,8 @@
namespace DB
{
class AnalyzedJoin;
namespace JoinStuff
{
@ -141,7 +143,7 @@ public:
/** Join data from the map (that was previously built by calls to insertFromBlock) to the block with data from "left" table.
* Could be called from different threads in parallel.
*/
void joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const;
void joinBlock(Block & block, const AnalyzedJoin & join_params) const;
/// Infer the return type for joinGet function
DataTypePtr joinGetReturnType(const String & column_name) const;
@ -161,8 +163,8 @@ public:
* Use only after all calls to joinBlock was done.
* left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside).
*/
BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, const Names & key_names_left,
const NamesAndTypesList & columns_added_by_join, UInt64 max_block_size) const;
BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, const AnalyzedJoin & join_params,
UInt64 max_block_size) const;
/// Number of keys in all built JOIN maps.
size_t getTotalRowCount() const;

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);
@ -489,14 +498,13 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const
}
}
void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin::Kind & join_kind)
void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin & out_table_join)
{
const ASTTablesInSelectQueryElement * node = select_query.join();
if (!node)
return;
auto & table_join = const_cast<ASTTablesInSelectQueryElement *>(node)->table_join->as<ASTTableJoin &>();
join_kind = table_join.kind;
if (table_join.strictness == ASTTableJoin::Strictness::Unspecified &&
table_join.kind != ASTTableJoin::Kind::Cross)
@ -509,6 +517,8 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul
throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty",
DB::ErrorCodes::EXPECTED_ALL_OR_ANY);
}
out_table_join = table_join;
}
/// Find the columns that are obtained by JOIN.
@ -609,8 +619,7 @@ std::vector<const ASTFunction *> getAggregates(const ASTPtr & query)
/// Calculate which columns are required to execute the expression.
/// Then, delete all other columns from the list of available columns.
/// After execution, columns will only contain the list of columns needed to read from the table.
void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns,
bool make_joined_columns_nullable)
void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns)
{
/// We caclulate required_source_columns with source_columns modifications and swap them on exit
required_source_columns = source_columns;
@ -637,8 +646,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA
avaliable_columns.insert(name.name);
/// Add columns obtained by JOIN (if needed).
columns_added_by_join.clear();
for (const auto & joined_column : analyzed_join.columns_from_joined_table)
for (const auto & joined_column : analyzed_join->columnsFromJoinedTable())
{
auto & name = joined_column.name;
if (avaliable_columns.count(name))
@ -647,16 +655,9 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA
if (required.count(name))
{
/// Optimisation: do not add columns needed only in JOIN ON section.
if (columns_context.nameInclusion(name) > analyzed_join.rightKeyInclusion(name))
{
if (make_joined_columns_nullable)
{
auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type;
columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type)));
}
else
columns_added_by_join.push_back(joined_column);
}
if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name))
analyzed_join->addJoinedColumn(joined_column);
required.erase(name);
}
}
@ -766,7 +767,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA
if (columns_context.has_table_join)
{
ss << ", joined columns:";
for (const auto & column : analyzed_join.columns_from_joined_table)
for (const auto & column : analyzed_join->columnsFromJoinedTable())
ss << " '" << column.name << "'";
}
@ -798,15 +799,17 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
storage = context.tryGetTable(db_and_table->database, db_and_table->table);
}
const auto & settings = context.getSettingsRef();
SyntaxAnalyzerResult result;
result.storage = storage;
result.source_columns = source_columns_;
result.analyzed_join = std::make_shared<AnalyzedJoin>(); /// TODO: move to select_query logic
result.analyzed_join->join_use_nulls = settings.join_use_nulls;
collectSourceColumns(select_query, result.storage, result.source_columns);
NameSet source_columns_set = removeDuplicateColumns(result.source_columns);
const auto & settings = context.getSettingsRef();
Names source_columns_list;
source_columns_list.reserve(result.source_columns.size());
for (const auto & type_name : result.source_columns)
@ -831,13 +834,13 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
const auto & joined_expression = node->table_expression->as<ASTTableExpression &>();
DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase());
result.analyzed_join.columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
result.analyzed_join.deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix());
result.analyzed_join->columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context);
result.analyzed_join->deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix());
}
translateQualifiedNames(query, *select_query, context,
(storage ? storage->getColumns().getOrdinary().getNames() : source_columns_list), source_columns_set,
result.analyzed_join.getQualifiedColumnsSet());
result.analyzed_join->getQualifiedColumnsSet());
/// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting.
InJoinSubqueriesPreprocessor(context).visit(query);
@ -872,7 +875,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
/// Optimize if with constant condition after constants was substituted instead of scalar subqueries.
OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query);
bool make_joined_columns_nullable = false;
if (select_query)
{
/// GROUP BY injective function elimination.
@ -893,15 +895,12 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze(
/// Push the predicate expression down to the subqueries.
result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize();
ASTTableJoin::Kind join_kind = ASTTableJoin::Kind::Comma;
setJoinStrictness(*select_query, settings.join_default_strictness, join_kind);
make_joined_columns_nullable = settings.join_use_nulls && isLeftOrFull(join_kind);
collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases);
setJoinStrictness(*select_query, settings.join_default_strictness, result.analyzed_join->table_join);
collectJoinedColumns(*result.analyzed_join, *select_query, source_columns_set, result.aliases);
}
result.aggregates = getAggregates(query);
result.collectUsedColumns(query, additional_source_columns, make_joined_columns_nullable);
result.collectUsedColumns(query, additional_source_columns);
return std::make_shared<const SyntaxAnalyzerResult>(result);
}

View File

@ -15,13 +15,11 @@ class ASTFunction;
struct SyntaxAnalyzerResult
{
StoragePtr storage;
AnalyzedJoin analyzed_join;
std::shared_ptr<AnalyzedJoin> analyzed_join;
NamesAndTypesList source_columns;
/// Set of columns that are enough to read from the table to evaluate the expression. It does not include joined columns.
NamesAndTypesList required_source_columns;
/// Columns will be added to block by JOIN. It's a subset of analyzed_join.columns_from_joined_table with corrected Nullability
NamesAndTypesList columns_added_by_join;
Aliases aliases;
std::vector<const ASTFunction *> aggregates;
@ -42,7 +40,7 @@ struct SyntaxAnalyzerResult
/// Predicate optimizer overrides the sub queries
bool rewrite_subqueries = false;
void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns, bool make_joined_columns_nullable);
void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns);
Names requiredSourceColumns() const { return required_source_columns.getNames(); }
};

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();
@ -193,10 +220,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;
}
@ -206,6 +235,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.
@ -218,7 +249,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);
@ -232,7 +265,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());
}
@ -324,7 +358,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();
@ -470,7 +504,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

@ -287,12 +287,6 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa
switch (node.last_processor_status)
{
case IProcessor::Status::NeedData:
{
add_neighbours_to_prepare_queue();
try_release_ownership();
break;
}
case IProcessor::Status::PortFull:
{
add_neighbours_to_prepare_queue();

View File

@ -174,7 +174,7 @@ Chunk IRowInputFormat::generate()
{
if (params.allow_errors_num > 0 || params.allow_errors_ratio > 0)
{
Logger * log = &Logger::get("BlockInputStreamFromRowInputStream");
Logger * log = &Logger::get("IRowInputFormat");
LOG_TRACE(log, "Skipped " << num_errors << " rows with errors while reading the input stream");
}

View File

@ -13,6 +13,8 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Interpreters/castColumn.h>
#include <algorithm>
namespace DB
{
@ -27,34 +29,28 @@ namespace DB
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
extern const int THERE_IS_NO_COLUMN;
}
const std::unordered_map<arrow::Type::type, std::shared_ptr<IDataType>> arrow_type_to_internal_type = {
//{arrow::Type::DECIMAL, std::make_shared<DataTypeDecimal>()},
{arrow::Type::UINT8, std::make_shared<DataTypeUInt8>()},
{arrow::Type::INT8, std::make_shared<DataTypeInt8>()},
{arrow::Type::UINT16, std::make_shared<DataTypeUInt16>()},
{arrow::Type::INT16, std::make_shared<DataTypeInt16>()},
{arrow::Type::UINT32, std::make_shared<DataTypeUInt32>()},
{arrow::Type::INT32, std::make_shared<DataTypeInt32>()},
{arrow::Type::UINT64, std::make_shared<DataTypeUInt64>()},
{arrow::Type::INT64, std::make_shared<DataTypeInt64>()},
{arrow::Type::HALF_FLOAT, std::make_shared<DataTypeFloat32>()},
{arrow::Type::FLOAT, std::make_shared<DataTypeFloat32>()},
{arrow::Type::DOUBLE, std::make_shared<DataTypeFloat64>()},
{arrow::Type::BOOL, std::make_shared<DataTypeUInt8>()},
//{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
//{arrow::Type::DATE32, std::make_shared<DataTypeDateTime>()},
{arrow::Type::DATE64, std::make_shared<DataTypeDateTime>()},
{arrow::Type::TIMESTAMP, std::make_shared<DataTypeDateTime>()},
//{arrow::Type::TIME32, std::make_shared<DataTypeDateTime>()},
static const std::initializer_list<std::pair<arrow::Type::type, const char *>> arrow_type_to_internal_type =
{
{arrow::Type::UINT8, "UInt8"},
{arrow::Type::INT8, "Int8"},
{arrow::Type::UINT16, "UInt16"},
{arrow::Type::INT16, "Int16"},
{arrow::Type::UINT32, "UInt32"},
{arrow::Type::INT32, "Int32"},
{arrow::Type::UINT64, "UInt64"},
{arrow::Type::INT64, "Int64"},
{arrow::Type::HALF_FLOAT, "Float32"},
{arrow::Type::FLOAT, "Float32"},
{arrow::Type::DOUBLE, "Float64"},
{arrow::Type::BOOL, "UInt8"},
{arrow::Type::DATE32, "Date"},
{arrow::Type::DATE64, "DateTime"},
{arrow::Type::TIMESTAMP, "DateTime"},
{arrow::Type::STRING, std::make_shared<DataTypeString>()},
{arrow::Type::BINARY, std::make_shared<DataTypeString>()},
//{arrow::Type::FIXED_SIZE_BINARY, std::make_shared<DataTypeString>()},
//{arrow::Type::UUID, std::make_shared<DataTypeString>()},
{arrow::Type::STRING, "String"},
{arrow::Type::BINARY, "String"},
// TODO: add other types that are convertable to internal ones:
// 0. ENUM?
@ -253,7 +249,7 @@ namespace DB
void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr<arrow::Table> &table,
arrow::Status &read_status, const Block &header,
int &row_group_current, const Context &context, std::string format_name)
{
{
Columns columns_list;
UInt64 num_rows = 0;
@ -308,15 +304,16 @@ namespace DB
const auto decimal_type = static_cast<arrow::DecimalType *>(arrow_column->type().get());
internal_nested_type = std::make_shared<DataTypeDecimal<Decimal128>>(decimal_type->precision(),
decimal_type->scale());
} else if (arrow_type_to_internal_type.find(arrow_type) != arrow_type_to_internal_type.end())
}
else if (auto internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(),
[=](auto && elem) { return elem.first == arrow_type; });
internal_type_it != arrow_type_to_internal_type.end())
{
internal_nested_type = arrow_type_to_internal_type.at(arrow_type);
internal_nested_type = DataTypeFactory::instance().get(internal_type_it->second);
}
else
{
throw Exception
{
"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name()
throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name()
+ "\" is not supported for conversion from a " + format_name + " data format",
ErrorCodes::CANNOT_CONVERT_TYPE};
}

View File

@ -5,6 +5,7 @@
#include <Processors/Formats/Impl/CSVRowInputFormat.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeNothing.h>
namespace DB
@ -17,9 +18,9 @@ namespace ErrorCodes
}
CSVRowInputFormat::CSVRowInputFormat(
ReadBuffer & in_, Block header_, Params params_, bool with_names_, const FormatSettings & format_settings_)
: IRowInputFormat(std::move(header_), in_, std::move(params_))
CSVRowInputFormat::CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_)
, with_names(with_names_)
, format_settings(format_settings_)
{
@ -79,72 +80,72 @@ void CSVRowInputFormat::addInputColumn(const String & column_name)
column_indexes_for_input_fields.emplace_back(column_index);
}
static void skipEndOfLine(ReadBuffer & istr)
static void skipEndOfLine(ReadBuffer & in)
{
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
if (*istr.position() == '\n')
if (*in.position() == '\n')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\r')
++istr.position();
++in.position();
if (!in.eof() && *in.position() == '\r')
++in.position();
}
else if (*istr.position() == '\r')
else if (*in.position() == '\r')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\n')
++istr.position();
++in.position();
if (!in.eof() && *in.position() == '\n')
++in.position();
else
throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)."
" Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA);
}
else if (!istr.eof())
else if (!in.eof())
throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA);
}
static void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column)
static void skipDelimiter(ReadBuffer & in, const char delimiter, bool is_last_column)
{
if (is_last_column)
{
if (istr.eof())
if (in.eof())
return;
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
if (*in.position() == delimiter)
{
++istr.position();
if (istr.eof())
++in.position();
if (in.eof())
return;
}
skipEndOfLine(istr);
skipEndOfLine(in);
}
else
assertChar(delimiter, istr);
assertChar(delimiter, in);
}
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
static inline void skipWhitespacesAndTabs(ReadBuffer & in)
{
while (!buf.eof()
&& (*buf.position() == ' '
|| *buf.position() == '\t'))
++buf.position();
while (!in.eof()
&& (*in.position() == ' '
|| *in.position() == '\t'))
++in.position();
}
static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns)
static void skipRow(ReadBuffer & in, const FormatSettings::CSV & settings, size_t num_columns)
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(istr);
readCSVString(tmp, istr, settings);
skipWhitespacesAndTabs(istr);
skipWhitespacesAndTabs(in);
readCSVString(tmp, in, settings);
skipWhitespacesAndTabs(in);
skipDelimiter(istr, settings.delimiter, i + 1 == num_columns);
skipDelimiter(in, settings.delimiter, i + 1 == num_columns);
}
}
@ -156,7 +157,6 @@ void CSVRowInputFormat::readPrefix()
skipBOMIfExists(in);
size_t num_columns = data_types.size();
String tmp;
auto & header = getPort().getHeader();
if (with_names)
@ -224,8 +224,7 @@ bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
const auto & table_column = column_indexes_for_input_fields[file_column];
const bool is_last_file_column =
file_column + 1 == column_indexes_for_input_fields.size();
const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size();
if (table_column)
{
@ -267,71 +266,7 @@ bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext
return true;
}
String CSVRowInputFormat::getDiagnosticInfo()
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
auto & header = getPort().getHeader();
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
in.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
/** gcc-7 generates wrong code with optimization level greater than 1.
* See tests: dbms/src/IO/tests/write_int.cpp
* and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh
* This is compiler bug. The bug does not present in gcc-8 and clang-8.
* Nevertheless, we don't need high optimization of this function.
*/
bool OPTIMIZE(1) CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
bool CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
const char delimiter = format_settings.csv.delimiter;
@ -345,100 +280,19 @@ bool OPTIMIZE(1) CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumn
if (column_indexes_for_input_fields[file_column].has_value())
{
const auto & table_column = *column_indexes_for_input_fields[file_column];
const auto & current_column_type = data_types[table_column];
const bool is_last_file_column =
file_column + 1 == column_indexes_for_input_fields.size();
const bool at_delimiter = !in.eof() && *in.position() == delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (in.eof() || *in.position() == '\n' || *in.position() == '\r');
auto & header = getPort().getHeader();
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(table_column).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(table_column).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
if (format_settings.csv.empty_as_default
&& (at_delimiter || at_last_column_line_end))
{
columns[table_column]->insertDefault();
}
else
{
BufferBase::Position prev_position = in.position();
BufferBase::Position curr_position = in.position();
std::exception_ptr exception;
try
{
skipWhitespacesAndTabs(in);
prev_position = in.position();
readField(*columns[table_column], current_column_type, is_last_file_column, table_column);
curr_position = in.position();
skipWhitespacesAndTabs(in);
}
catch (...)
{
exception = std::current_exception();
}
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue()
&& *curr_position != '\n' && *curr_position != '\r'
&& *curr_position != delimiter)
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
size_t col_idx = column_indexes_for_input_fields[file_column].value();
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx],
out, file_column))
return false;
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
String tmp;
readCSVString(tmp, in, format_settings.csv);
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column))
return false;
}
/// Delimiters
@ -502,15 +356,26 @@ void CSVRowInputFormat::syncAfterError()
skipToNextLineOrEOF(in);
}
void CSVRowInputFormat::updateDiagnosticInfo()
void CSVRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos)
{
++row_num;
skipWhitespacesAndTabs(in);
prev_pos = in.position();
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset();
if (column_indexes_for_input_fields[file_column])
{
const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size();
if (!readField(column, type, is_last_file_column, *column_indexes_for_input_fields[file_column]))
column.insertDefault();
}
else
{
String tmp;
readCSVString(tmp, in, format_settings.csv);
}
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = in.position();
curr_pos = in.position();
skipWhitespacesAndTabs(in);
}
bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx)
@ -563,7 +428,7 @@ void registerInputFormatProcessorCSV(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<CSVRowInputFormat>(buf, sample, std::move(params), with_names, settings);
return std::make_shared<CSVRowInputFormat>(sample, buf, params, with_names, settings);
});
}
}

View File

@ -1,40 +1,38 @@
#pragma once
#include <optional>
#include <unordered_map>
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class ReadBuffer;
/** A stream for inputting data in csv format.
* Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values.
*/
class CSVRowInputFormat : public IRowInputFormat
class CSVRowInputFormat : public RowInputFormatWithDiagnosticInfo
{
public:
/** with_names - in the first line the header with column names
* with_types - on the next line header with type names
*/
CSVRowInputFormat(ReadBuffer & in_, Block header_, Params params_, bool with_names_, const FormatSettings & format_settings_);
CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, const FormatSettings & format_settings_);
String getName() const override { return "CSVRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension &) override;
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
bool with_names;
DataTypes data_types;
const FormatSettings format_settings;
DataTypes data_types;
using IndexesMap = std::unordered_map<String, size_t>;
IndexesMap column_indexes_by_names;
@ -43,7 +41,7 @@ private:
using OptionalIndexes = std::vector<std::optional<size_t>>;
OptionalIndexes column_indexes_for_input_fields;
/// Tracks which colums we have read in a single read() call.
/// Tracks which columns we have read in a single read() call.
/// For columns that are never read, it is initialized to false when we
/// read the file header, and never changed afterwards.
/// For other columns, it is updated on each read() call.
@ -55,26 +53,19 @@ private:
void addInputColumn(const String & column_name);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those that are still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override
{
return *pos != '\n' && *pos != '\r' && *pos != format_settings.csv.delimiter;
}
/// For setting input_format_null_as_default
DataTypes nullable_types;
MutableColumns nullable_columns;
OptionalIndexes column_idx_to_nullable_column_idx;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
bool readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx);
};

View File

@ -5,6 +5,7 @@
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeNothing.h>
namespace DB
{
@ -16,23 +17,23 @@ namespace ErrorCodes
}
static void skipTSVRow(ReadBuffer & istr, const size_t num_columns)
static void skipTSVRow(ReadBuffer & in, const size_t num_columns)
{
NullSink null_sink;
for (size_t i = 0; i < num_columns; ++i)
{
readEscapedStringInto(null_sink, istr);
assertChar(i == num_columns - 1 ? '\n' : '\t', istr);
readEscapedStringInto(null_sink, in);
assertChar(i == num_columns - 1 ? '\n' : '\t', in);
}
}
/** Check for a common error case - usage of Windows line feed.
*/
static void checkForCarriageReturn(ReadBuffer & istr)
static void checkForCarriageReturn(ReadBuffer & in)
{
if (istr.position()[0] == '\r' || (istr.position() != istr.buffer().begin() && istr.position()[-1] == '\r'))
if (in.position()[0] == '\r' || (in.position() != in.buffer().begin() && in.position()[-1] == '\r'))
throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row."
"\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format."
" You must transform your file to Unix format."
@ -41,9 +42,9 @@ static void checkForCarriageReturn(ReadBuffer & istr)
}
TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(
ReadBuffer & in_, Block header_, bool with_names_, bool with_types_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(std::move(header_), in_, std::move(params_)), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_)
{
auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
@ -173,9 +174,9 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
updateDiagnosticInfo();
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
const auto & column_index = column_indexes_for_input_fields[input_position];
const auto & column_index = column_indexes_for_input_fields[file_column];
if (column_index)
{
data_types[*column_index]->deserializeAsTextEscaped(*columns[*column_index], in, format_settings);
@ -187,7 +188,7 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
}
/// skip separators
if (input_position + 1 < column_indexes_for_input_fields.size())
if (file_column + 1 < column_indexes_for_input_fields.size())
{
assertChar('\t', in);
}
@ -205,160 +206,35 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens
return true;
}
String TabSeparatedRowInputFormat::getDiagnosticInfo()
bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
auto & header = getPort().getHeader();
WriteBufferFromOwnString out;
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current line and pars all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
in.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position)
{
if (input_position == 0 && in.eof())
if (file_column == 0 && in.eof())
{
out << "<End of stream>\n";
return false;
}
if (column_indexes_for_input_fields[input_position].has_value())
if (column_indexes_for_input_fields[file_column].has_value())
{
const auto & column_index = *column_indexes_for_input_fields[input_position];
const auto & current_column_type = data_types[column_index];
const auto & header = getPort().getHeader();
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(column_index).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(column_index).name.size(), ' ')
<< "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' ');
auto prev_position = in.position();
std::exception_ptr exception;
try
{
current_column_type->deserializeAsTextEscaped(*columns[column_index], in, format_settings);
}
catch (...)
{
exception = std::current_exception();
}
auto curr_position = in.position();
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << current_column_type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
auto & header = getPort().getHeader();
size_t col_idx = column_indexes_for_input_fields[file_column].value();
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx],
out, file_column))
return false;
}
out << "\n";
if (current_column_type->haveMaximumSizeOfValue())
{
if (*curr_position != '\n' && *curr_position != '\t')
{
out << "ERROR: garbage after " << current_column_type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (current_column_type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (current_column_type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ')
<< "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ')
<< "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' ');
NullSink null_sink;
readEscapedStringInto(null_sink, in);
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column))
return false;
}
/// Delimiters
if (input_position + 1 == column_indexes_for_input_fields.size())
if (file_column + 1 == column_indexes_for_input_fields.size())
{
if (!in.eof())
{
@ -401,7 +277,8 @@ bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns &
{
out << "ERROR: Line feed found where tab is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n";
"And if your file have right number of columns, "
"maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n";
}
else if (*in.position() == '\r')
{
@ -421,6 +298,19 @@ bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns &
return true;
}
void TabSeparatedRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos)
{
prev_pos = in.position();
if (column_indexes_for_input_fields[file_column])
type->deserializeAsTextEscaped(column, in, format_settings);
else
{
NullSink null_sink;
readEscapedStringInto(null_sink, in);
}
curr_pos = in.position();
}
void TabSeparatedRowInputFormat::syncAfterError()
{
@ -428,18 +318,6 @@ void TabSeparatedRowInputFormat::syncAfterError()
}
void TabSeparatedRowInputFormat::updateDiagnosticInfo()
{
++row_num;
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = in.position();
}
void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
{
for (auto name : {"TabSeparated", "TSV"})
@ -451,7 +329,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, false, false, std::move(params), settings);
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, false, false, settings);
});
}
@ -464,7 +342,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, true, false, std::move(params), settings);
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, true, false, settings);
});
}
@ -477,7 +355,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, true, true, std::move(params), settings);
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, params, true, true, settings);
});
}
}

View File

@ -2,25 +2,22 @@
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
namespace DB
{
class ReadBuffer;
/** A stream to input data in tsv format.
*/
class TabSeparatedRowInputFormat : public IRowInputFormat
class TabSeparatedRowInputFormat : public RowInputFormatWithDiagnosticInfo
{
public:
/** with_names - the first line is the header with the names of the columns
* with_types - on the next line header with type names
*/
TabSeparatedRowInputFormat(
ReadBuffer & in_, Block header_, bool with_names_, bool with_types_, Params params_, const FormatSettings & format_settings_);
TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_);
String getName() const override { return "TabSeparatedRowInputFormat"; }
@ -29,8 +26,6 @@ public:
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
bool with_names;
bool with_types;
@ -50,21 +45,10 @@ private:
void setupAllColumnsByTableSchema();
void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension& ext);
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; }
};
}

View File

@ -0,0 +1,243 @@
#include <Processors/Formats/Impl/TemplateBlockOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_)
: IOutputFormat(header_, out_), settings(settings_)
{
auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
types.resize(columns);
for (size_t i = 0; i < columns; ++i)
types[i] = sample.safeGetByPosition(i).type;
/// Parse format string for whole output
static const String default_format("${data}");
const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format;
format = ParsedTemplateFormatString(format_str, [&](const String & partName)
{
return static_cast<size_t>(stringToOutputPart(partName));
});
/// Validate format string for whole output
size_t data_idx = format.format_idx_to_column_idx.size() + 1;
for (size_t i = 0; i < format.format_idx_to_column_idx.size(); ++i)
{
if (!format.format_idx_to_column_idx[i])
format.throwInvalidFormat("Output part name cannot be empty, it's a bug.", i);
switch (static_cast<OutputPart>(*format.format_idx_to_column_idx[i]))
{
case OutputPart::Data:
data_idx = i;
[[fallthrough]];
case OutputPart::Totals:
case OutputPart::ExtremesMin:
case OutputPart::ExtremesMax:
if (format.formats[i] != ColumnFormat::None)
format.throwInvalidFormat("Serialization type for data, totals, min and max must be empty or None", i);
break;
default:
if (format.formats[i] == ColumnFormat::None)
format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, "
"rows_read or bytes_read is not specified", i);
break;
}
}
if (data_idx != 0)
format.throwInvalidFormat("${data} must be the first output part", 0);
/// Parse format string for rows
row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName)
{
return sample.getPositionByName(colName);
});
/// Validate format string for rows
if (row_format.delimiters.size() == 1)
row_format.throwInvalidFormat("No columns specified", 0);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (!row_format.format_idx_to_column_idx[i])
row_format.throwInvalidFormat("Cannot skip format field for output, it's a bug.", i);
if (row_format.formats[i] == ColumnFormat::None)
row_format.throwInvalidFormat("Serialization type for file column is not specified", i);
}
}
TemplateBlockOutputFormat::OutputPart TemplateBlockOutputFormat::stringToOutputPart(const String & part)
{
if (part == "data")
return OutputPart::Data;
else if (part == "totals")
return OutputPart::Totals;
else if (part == "min")
return OutputPart::ExtremesMin;
else if (part == "max")
return OutputPart::ExtremesMax;
else if (part == "rows")
return OutputPart::Rows;
else if (part == "rows_before_limit")
return OutputPart::RowsBeforeLimit;
else if (part == "time")
return OutputPart::TimeElapsed;
else if (part == "rows_read")
return OutputPart::RowsRead;
else if (part == "bytes_read")
return OutputPart::BytesRead;
else
throw Exception("Unknown output part " + part, ErrorCodes::SYNTAX_ERROR);
}
void TemplateBlockOutputFormat::writeRow(const Chunk & chunk, size_t row_num)
{
size_t columns = row_format.format_idx_to_column_idx.size();
for (size_t j = 0; j < columns; ++j)
{
writeString(row_format.delimiters[j], out);
size_t col_idx = *row_format.format_idx_to_column_idx[j];
serializeField(*chunk.getColumns()[col_idx], *types[col_idx], row_num, row_format.formats[j]);
}
writeString(row_format.delimiters[columns], out);
}
void TemplateBlockOutputFormat::serializeField(const IColumn & column, const IDataType & type, size_t row_num, ColumnFormat col_format)
{
switch (col_format)
{
case ColumnFormat::Escaped:
type.serializeAsTextEscaped(column, row_num, out, settings);
break;
case ColumnFormat::Quoted:
type.serializeAsTextQuoted(column, row_num, out, settings);
break;
case ColumnFormat::Csv:
type.serializeAsTextCSV(column, row_num, out, settings);
break;
case ColumnFormat::Json:
type.serializeAsTextJSON(column, row_num, out, settings);
break;
case ColumnFormat::Xml:
type.serializeAsTextXML(column, row_num, out, settings);
break;
case ColumnFormat::Raw:
type.serializeAsText(column, row_num, out, settings);
break;
default:
__builtin_unreachable();
}
}
template <typename U, typename V> void TemplateBlockOutputFormat::writeValue(U value, ColumnFormat col_format)
{
auto type = std::make_unique<V>();
auto col = type->createColumn();
col->insert(value);
serializeField(*col, *type, 0, col_format);
}
void TemplateBlockOutputFormat::consume(Chunk chunk)
{
doWritePrefix();
size_t rows = chunk.getNumRows();
for (size_t i = 0; i < rows; ++i)
{
if (row_count)
writeString(settings.template_settings.row_between_delimiter, out);
writeRow(chunk, i);
++row_count;
}
}
void TemplateBlockOutputFormat::doWritePrefix()
{
if (need_write_prefix)
{
writeString(format.delimiters.front(), out);
need_write_prefix = false;
}
}
void TemplateBlockOutputFormat::finalize()
{
if (finalized)
return;
doWritePrefix();
size_t parts = format.format_idx_to_column_idx.size();
for (size_t i = 0; i < parts; ++i)
{
auto type = std::make_shared<DataTypeUInt64>();
ColumnWithTypeAndName col(type->createColumnConst(1, row_count), type, String("tmp"));
switch (static_cast<OutputPart>(*format.format_idx_to_column_idx[i]))
{
case OutputPart::Totals:
if (!totals)
format.throwInvalidFormat("Cannot print totals for this request", i);
writeRow(totals, 0);
break;
case OutputPart::ExtremesMin:
if (!extremes)
format.throwInvalidFormat("Cannot print extremes for this request", i);
writeRow(extremes, 0);
break;
case OutputPart::ExtremesMax:
if (!extremes)
format.throwInvalidFormat("Cannot print extremes for this request", i);
writeRow(extremes, 1);
break;
case OutputPart::Rows:
writeValue<size_t, DataTypeUInt64>(row_count, format.formats[i]);
break;
case OutputPart::RowsBeforeLimit:
if (!rows_before_limit_set)
format.throwInvalidFormat("Cannot print rows_before_limit for this request", i);
writeValue<size_t, DataTypeUInt64>(rows_before_limit, format.formats[i]);
break;
case OutputPart::TimeElapsed:
writeValue<double, DataTypeFloat64>(watch.elapsedSeconds(), format.formats[i]);
break;
case OutputPart::RowsRead:
writeValue<size_t, DataTypeUInt64>(progress.read_rows.load(), format.formats[i]);
break;
case OutputPart::BytesRead:
writeValue<size_t, DataTypeUInt64>(progress.read_bytes.load(), format.formats[i]);
break;
default:
break;
}
writeString(format.delimiters[i + 1], out);
}
finalized = true;
}
void registerOutputFormatProcessorTemplate(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("Template", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
FormatFactory::WriteCallback,
const FormatSettings & settings)
{
return std::make_shared<TemplateBlockOutputFormat>(sample, buf, settings);
});
}
}

View File

@ -0,0 +1,68 @@
#pragma once
#include <Common/Stopwatch.h>
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/ParsedTemplateFormatString.h>
namespace DB
{
class TemplateBlockOutputFormat : public IOutputFormat
{
using ColumnFormat = ParsedTemplateFormatString::ColumnFormat;
public:
TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_);
String getName() const override { return "TemplateBlockOutputFormat"; }
void doWritePrefix() override;
void setRowsBeforeLimit(size_t rows_before_limit_) override { rows_before_limit = rows_before_limit_; rows_before_limit_set = true; }
void onProgress(const Progress & progress_) override { progress.incrementPiecewiseAtomically(progress_); }
protected:
void consume(Chunk chunk) override;
void consumeTotals(Chunk chunk) override { totals = std::move(chunk); }
void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); }
void finalize() override;
enum class OutputPart : size_t
{
Data,
Totals,
ExtremesMin,
ExtremesMax,
Rows,
RowsBeforeLimit,
TimeElapsed,
RowsRead,
BytesRead
};
OutputPart stringToOutputPart(const String & part);
void writeRow(const Chunk & chunk, size_t row_num);
void serializeField(const IColumn & column, const IDataType & type, size_t row_num, ColumnFormat format);
template <typename U, typename V> void writeValue(U value, ColumnFormat col_format);
protected:
const FormatSettings settings;
DataTypes types;
ParsedTemplateFormatString format;
ParsedTemplateFormatString row_format;
size_t rows_before_limit = 0;
bool rows_before_limit_set = false;
Chunk totals;
Chunk extremes;
Progress progress;
Stopwatch watch;
size_t row_count = 0;
bool need_write_prefix = true;
};
}

View File

@ -0,0 +1,520 @@
#include <Processors/Formats/Impl/TemplateRowInputFormat.h>
#include <Formats/FormatFactory.h>
#include <Formats/verbosePrintString.h>
#include <IO/Operators.h>
#include <DataTypes/DataTypeNothing.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ATTEMPT_TO_READ_AFTER_EOF;
extern const int CANNOT_READ_ALL_DATA;
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int SYNTAX_ERROR;
}
TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
const FormatSettings & settings_, bool ignore_spaces_)
: RowInputFormatWithDiagnosticInfo(header_, buf, params_), buf(in_), data_types(header_.getDataTypes()),
settings(settings_), ignore_spaces(ignore_spaces_)
{
/// Parse format string for whole input
static const String default_format("${data}");
const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format;
format = ParsedTemplateFormatString(format_str, [&](const String & partName) -> std::optional<size_t>
{
if (partName == "data")
return 0;
else if (partName.empty()) /// For skipping some values in prefix and suffix
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
/// Suppress false-positive warning (bug in GCC 9: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=86465)
return {};
#if !__clang__
#pragma GCC diagnostic pop
#endif
throw Exception("Unknown input part " + partName, ErrorCodes::SYNTAX_ERROR);
});
/// Validate format string for whole input
bool has_data = false;
for (size_t i = 0; i < format.columnsCount(); ++i)
{
if (format.format_idx_to_column_idx[i])
{
if (has_data)
format.throwInvalidFormat("${data} can occur only once", i);
if (format.formats[i] != ColumnFormat::None)
format.throwInvalidFormat("${data} must have empty or None deserialization type", i);
has_data = true;
format_data_idx = i;
}
else
{
if (format.formats[i] == ColumnFormat::Xml || format.formats[i] == ColumnFormat::Raw)
format.throwInvalidFormat("XML and Raw deserialization is not supported", i);
}
}
/// Parse format string for rows
row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName) -> std::optional<size_t>
{
if (colName.empty())
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
return {};
#if !__clang__
#pragma GCC diagnostic pop
#endif
return header_.getPositionByName(colName);
});
/// Validate format string for rows
std::vector<UInt8> column_in_format(header_.columns(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (row_format.formats[i] == ColumnFormat::Xml || row_format.formats[i] == ColumnFormat::Raw)
row_format.throwInvalidFormat("XML and Raw deserialization is not supported", i);
if (row_format.format_idx_to_column_idx[i])
{
if (row_format.formats[i] == ColumnFormat::None)
row_format.throwInvalidFormat("Column is not skipped, but deserialization type is None", i);
size_t col_idx = *row_format.format_idx_to_column_idx[i];
if (column_in_format[col_idx])
row_format.throwInvalidFormat("Duplicate column", i);
column_in_format[col_idx] = true;
}
}
}
void TemplateRowInputFormat::readPrefix()
{
size_t last_successfully_parsed_idx = 0;
try
{
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format_data_idx);
}
catch (Exception & e)
{
format.throwInvalidFormat(e.message() + " While parsing prefix", last_successfully_parsed_idx);
}
}
/// Asserts delimiters and skips fields in prefix or suffix.
/// tryReadPrefixOrSuffix<bool>(...) is used in checkForSuffix() to avoid throwing an exception after read of each row
/// (most likely false will be returned on first call of checkString(...))
template <typename ReturnType>
ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], buf)))
return ReturnType(false);
}
while (input_part_beg < input_part_end)
{
skipSpaces();
if constexpr (throw_exception)
skipField(format.formats[input_part_beg]);
else
{
try
{
skipField(format.formats[input_part_beg]);
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
/// If it's parsing error, then suffix is not found
return ReturnType(false);
}
}
++input_part_beg;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], buf)))
return ReturnType(false);
}
}
if constexpr (!throw_exception)
return ReturnType(true);
}
bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & extra)
{
/// This function can be called again after it returned false
if (unlikely(end_of_stream))
return false;
skipSpaces();
if (unlikely(checkForSuffix()))
{
end_of_stream = true;
return false;
}
updateDiagnosticInfo();
if (likely(row_num != 1))
assertString(settings.template_settings.row_between_delimiter, buf);
extra.read_columns.assign(columns.size(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
skipSpaces();
assertString(row_format.delimiters[i], buf);
skipSpaces();
if (row_format.format_idx_to_column_idx[i])
{
size_t col_idx = *row_format.format_idx_to_column_idx[i];
deserializeField(*data_types[col_idx], *columns[col_idx], row_format.formats[i]);
extra.read_columns[col_idx] = true;
}
else
skipField(row_format.formats[i]);
}
skipSpaces();
assertString(row_format.delimiters.back(), buf);
for (size_t i = 0; i < columns.size(); ++i)
if (!extra.read_columns[i])
data_types[i]->insertDefaultInto(*columns[i]);
return true;
}
void TemplateRowInputFormat::deserializeField(const IDataType & type, IColumn & column, ColumnFormat col_format)
{
try
{
switch (col_format)
{
case ColumnFormat::Escaped:
type.deserializeAsTextEscaped(column, buf, settings);
break;
case ColumnFormat::Quoted:
type.deserializeAsTextQuoted(column, buf, settings);
break;
case ColumnFormat::Csv:
type.deserializeAsTextCSV(column, buf, settings);
break;
case ColumnFormat::Json:
type.deserializeAsTextJSON(column, buf, settings);
break;
default:
__builtin_unreachable();
}
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof();
throw;
}
}
void TemplateRowInputFormat::skipField(TemplateRowInputFormat::ColumnFormat col_format)
{
String tmp;
constexpr const char * field_name = "<SKIPPED COLUMN>";
constexpr size_t field_name_len = 16;
try
{
switch (col_format)
{
case ColumnFormat::None:
/// Empty field, just skip spaces
break;
case ColumnFormat::Escaped:
readEscapedString(tmp, buf);
break;
case ColumnFormat::Quoted:
readQuotedString(tmp, buf);
break;
case ColumnFormat::Csv:
readCSVString(tmp, buf, settings.csv);
break;
case ColumnFormat::Json:
skipJSONField(buf, StringRef(field_name, field_name_len));
break;
default:
__builtin_unreachable();
}
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof();
throw;
}
}
/// Returns true if all rows have been read i.e. there are only suffix and spaces (if ignore_spaces == true) before EOF.
/// Otherwise returns false
bool TemplateRowInputFormat::checkForSuffix()
{
PeekableReadBufferCheckpoint checkpoint{buf};
bool suffix_found = false;
size_t last_successfully_parsed_idx = format_data_idx + 1;
try
{
suffix_found = tryReadPrefixOrSuffix<bool>(last_successfully_parsed_idx, format.columnsCount());
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
}
if (unlikely(suffix_found))
{
skipSpaces();
if (buf.eof())
return true;
}
buf.rollbackToCheckpoint();
return false;
}
bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
out << "Suffix does not match: ";
size_t last_successfully_parsed_idx = format_data_idx + 1;
const ReadBuffer::Position row_begin_pos = buf.position();
bool caught = false;
try
{
PeekableReadBufferCheckpoint checkpoint{buf, true};
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format.columnsCount());
}
catch (Exception & e)
{
out << e.message() << " Near column " << last_successfully_parsed_idx;
caught = true;
}
if (!caught)
{
out << " There is some data after suffix (EOF expected, got ";
verbosePrintString(buf.position(), std::min(buf.buffer().end(), buf.position() + 16), out);
out << "). ";
}
out << " Format string (from format_schema): \n" << format.dump() << "\n";
if (row_begin_pos != buf.position())
{
/// Pointers to buffer memory were invalidated during checking for suffix
out << "\nCannot print more diagnostic info.";
return false;
}
out << "\nUsing format string (from format_schema_rows): " << row_format.dump() << "\n";
out << "\nTrying to parse next row, because suffix does not match:\n";
try
{
if (likely(row_num != 1))
assertString(settings.template_settings.row_between_delimiter, buf);
}
catch (const DB::Exception &)
{
writeErrorStringForWrongDelimiter(out, "delimiter between rows", settings.template_settings.row_between_delimiter);
return false;
}
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
skipSpaces();
try
{
assertString(row_format.delimiters[i], buf);
}
catch (const DB::Exception &)
{
writeErrorStringForWrongDelimiter(out, "delimiter before field " + std::to_string(i), row_format.delimiters[i]);
return false;
}
skipSpaces();
if (row_format.format_idx_to_column_idx[i])
{
auto & header = getPort().getHeader();
size_t col_idx = *row_format.format_idx_to_column_idx[i];
if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx],
*columns[col_idx], out, i))
{
out << "Maybe it's not possible to deserialize field " + std::to_string(i) +
" as " + ParsedTemplateFormatString::formatToString(row_format.formats[i]);
return false;
}
}
else
{
static const String skipped_column_str = "<SKIPPED COLUMN>";
static const DataTypePtr skipped_column_type = std::make_shared<DataTypeNothing>();
static const MutableColumnPtr skipped_column = skipped_column_type->createColumn();
if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, i))
return false;
}
}
skipSpaces();
try
{
assertString(row_format.delimiters.back(), buf);
}
catch (const DB::Exception &)
{
writeErrorStringForWrongDelimiter(out, "delimiter after last field", row_format.delimiters.back());
return false;
}
return true;
}
void TemplateRowInputFormat::writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim)
{
out << "ERROR: There is no " << description << ": expected ";
verbosePrintString(delim.data(), delim.data() + delim.size(), out);
out << ", got ";
if (buf.eof())
out << "<End of stream>";
else
verbosePrintString(buf.position(), std::min(buf.position() + delim.size() + 10, buf.buffer().end()), out);
out << '\n';
}
void TemplateRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos)
{
prev_pos = buf.position();
if (row_format.format_idx_to_column_idx[file_column])
deserializeField(*type, column, row_format.formats[file_column]);
else
skipField(row_format.formats[file_column]);
curr_pos = buf.position();
}
bool TemplateRowInputFormat::isGarbageAfterField(size_t, ReadBuffer::Position)
{
/// Garbage will be considered as wrong delimiter
return false;
}
bool TemplateRowInputFormat::allowSyncAfterError() const
{
return !row_format.delimiters.back().empty() || !settings.template_settings.row_between_delimiter.empty();
}
void TemplateRowInputFormat::syncAfterError()
{
bool at_beginning_of_row_or_eof = false;
while (!at_beginning_of_row_or_eof)
{
skipToNextDelimiterOrEof(row_format.delimiters.back());
if (buf.eof())
{
end_of_stream = true;
return;
}
buf.ignore(row_format.delimiters.back().size());
skipSpaces();
if (checkForSuffix())
return;
bool last_delimiter_in_row_found = !row_format.delimiters.back().empty();
if (last_delimiter_in_row_found && checkString(settings.template_settings.row_between_delimiter, buf))
at_beginning_of_row_or_eof = true;
else
skipToNextDelimiterOrEof(settings.template_settings.row_between_delimiter);
if (buf.eof())
at_beginning_of_row_or_eof = end_of_stream = true;
}
/// It can happen that buf.position() is not at the beginning of row
/// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter.
/// It will cause another parsing error.
}
/// Searches for delimiter in input stream and sets buffer position to the beginning of delimiter (if found) or EOF (if not)
void TemplateRowInputFormat::skipToNextDelimiterOrEof(const String & delimiter)
{
if (delimiter.empty())
return;
while (!buf.eof())
{
void * pos = memchr(buf.position(), delimiter[0], buf.available());
if (!pos)
{
buf.position() += buf.available();
continue;
}
buf.position() = static_cast<ReadBuffer::Position>(pos);
PeekableReadBufferCheckpoint checkpoint{buf};
if (checkString(delimiter, buf))
return;
buf.rollbackToCheckpoint();
++buf.position();
}
}
void TemplateRowInputFormat::throwUnexpectedEof()
{
throw Exception("Unexpected EOF while parsing row " + std::to_string(row_num) + ". "
"Maybe last row has wrong format or input doesn't contain specified suffix before EOF.",
ErrorCodes::CANNOT_READ_ALL_DATA);
}
void registerInputFormatProcessorTemplate(FormatFactory & factory)
{
for (bool ignore_spaces : {false, true})
{
factory.registerInputFormatProcessor(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, settings, ignore_spaces);
});
}
}
}

View File

@ -0,0 +1,61 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Formats/FormatSettings.h>
#include <Formats/ParsedTemplateFormatString.h>
#include <IO/ReadHelpers.h>
#include <IO/PeekableReadBuffer.h>
namespace DB
{
class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo
{
using ColumnFormat = ParsedTemplateFormatString::ColumnFormat;
public:
TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_,
const FormatSettings & settings_, bool ignore_spaces_);
String getName() const override { return "TemplateRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension & extra) override;
void readPrefix() override;
bool allowSyncAfterError() const override;
void syncAfterError() override;
private:
void deserializeField(const IDataType & type, IColumn & column, ColumnFormat col_format);
void skipField(ColumnFormat col_format);
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(buf); }
template <typename ReturnType = void>
ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end);
bool checkForSuffix();
[[noreturn]] void throwUnexpectedEof();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, ReadBuffer::Position & prev_pos,
ReadBuffer::Position & curr_pos) override;
bool isGarbageAfterField(size_t after_col_idx, ReadBuffer::Position pos) override;
void writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim);
void skipToNextDelimiterOrEof(const String & delimiter);
private:
PeekableReadBuffer buf;
DataTypes data_types;
FormatSettings settings;
ParsedTemplateFormatString format;
ParsedTemplateFormatString row_format;
const bool ignore_spaces;
size_t format_data_idx;
bool end_of_stream = false;
};
}

View File

@ -0,0 +1,167 @@
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Formats/verbosePrintString.h>
#include <IO/Operators.h>
#include <IO/WriteBufferFromString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
DB::RowInputFormatWithDiagnosticInfo::RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_)
: IRowInputFormat(header_, in_, params_)
{
}
void DB::RowInputFormatWithDiagnosticInfo::updateDiagnosticInfo()
{
++row_num;
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset();
offset_of_prev_row = offset_of_current_row;
offset_of_current_row = in.offset();
}
String DB::RowInputFormatWithDiagnosticInfo::getDiagnosticInfo()
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
auto & header = getPort().getHeader();
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (offset_of_prev_row <= in.buffer().size())
{
in.position() = in.buffer().begin() + offset_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out))
return out.str();
}
else
{
if (in.buffer().size() < offset_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = in.buffer().begin() + offset_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out);
out << "\n";
return out.str();
}
bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(const String & col_name,
const DataTypePtr & type,
IColumn & column,
WriteBuffer & out,
size_t file_column)
{
out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ')
<< "name: " << alignedName(col_name, max_length_of_column_name)
<< "type: " << alignedName(type->getName(), max_length_of_data_type_name);
auto prev_position = in.position();
auto curr_position = in.position();
std::exception_ptr exception;
try
{
tryDeserializeFiled(type, column, file_column, prev_position, curr_position);
}
catch (...)
{
exception = std::current_exception();
}
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNativeNumber(type) || isDateOrDateTime(type))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << type->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (type->haveMaximumSizeOfValue())
{
if (isGarbageAfterField(file_column, curr_position))
{
out << "ERROR: garbage after " << type->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (type->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (type->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
return true;
}
String RowInputFormatWithDiagnosticInfo::alignedName(const String & name, size_t max_length) const
{
size_t spaces_count = max_length >= name.size() ? max_length - name.size() : 0;
return name + ", " + std::string(spaces_count, ' ');
}
}

View File

@ -0,0 +1,46 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <IO/ReadBuffer.h>
#include <limits>
namespace DB
{
class RowInputFormatWithDiagnosticInfo : public IRowInputFormat
{
public:
RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_);
String getDiagnosticInfo() override;
protected:
void updateDiagnosticInfo();
bool deserializeFieldAndPrintDiagnosticInfo(const String & col_name, const DataTypePtr & type, IColumn & column,
WriteBuffer & out, size_t file_column);
String alignedName(const String & name, size_t max_length) const;
virtual bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) = 0;
virtual void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column,
ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) = 0;
virtual bool isGarbageAfterField(size_t after_input_pos_idx, ReadBuffer::Position pos) = 0;
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
private:
/// How many bytes were read, not counting those still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
size_t offset_of_current_row = std::numeric_limits<size_t>::max();
size_t offset_of_prev_row = std::numeric_limits<size_t>::max();
/// For alignment of diagnostic info.
size_t max_length_of_column_name = 0;
size_t max_length_of_data_type_name = 0;
};
}

View File

@ -14,14 +14,28 @@ namespace ProfileEvents
namespace DB
{
static Chunk convertToChunk(const Block & block)
{
auto info = std::make_shared<AggregatedChunkInfo>();
info->bucket_num = block.info.bucket_num;
info->is_overflows = block.info.is_overflows;
UInt64 num_rows = block.rows();
Chunk chunk(block.getColumns(), num_rows);
chunk.setChunkInfo(std::move(info));
return chunk;
}
namespace
{
class SourceFromNativeStream : public ISource
{
public:
SourceFromNativeStream(const Block & header, const std::string & path)
: ISource(header), file_in(path), compressed_in(file_in)
, block_in(std::make_shared<NativeBlockInputStream>(compressed_in, ClickHouseRevision::get()))
: ISource(header), file_in(path), compressed_in(file_in),
block_in(std::make_shared<NativeBlockInputStream>(compressed_in, ClickHouseRevision::get()))
{
block_in->readPrefix();
}
@ -41,15 +55,7 @@ namespace
return {};
}
auto info = std::make_shared<AggregatedChunkInfo>();
info->bucket_num = block.info.bucket_num;
info->is_overflows = block.info.is_overflows;
UInt64 num_rows = block.rows();
Chunk chunk(block.getColumns(), num_rows);
chunk.setChunkInfo(std::move(info));
return chunk;
return convertToChunk(block);
}
private:
@ -57,39 +63,320 @@ namespace
CompressedReadBuffer compressed_in;
BlockInputStreamPtr block_in;
};
}
class ConvertingAggregatedToBlocksTransform : public ISource
class ConvertingAggregatedToChunksSource : public ISource
{
public:
ConvertingAggregatedToChunksSource(
AggregatingTransformParamsPtr params_,
ManyAggregatedDataVariantsPtr data_,
Arena * arena_,
std::shared_ptr<std::atomic<UInt32>> next_bucket_to_merge_)
: ISource(params_->getHeader())
, params(std::move(params_))
, data(std::move(data_))
, next_bucket_to_merge(std::move(next_bucket_to_merge_))
, arena(arena_)
{}
String getName() const override { return "ConvertingAggregatedToChunksSource"; }
protected:
Chunk generate() override
{
public:
ConvertingAggregatedToBlocksTransform(Block header, AggregatingTransformParamsPtr params_, BlockInputStreamPtr stream_)
: ISource(std::move(header)), params(std::move(params_)), stream(std::move(stream_)) {}
UInt32 bucket_num = next_bucket_to_merge->fetch_add(1);
String getName() const override { return "ConvertingAggregatedToBlocksTransform"; }
if (bucket_num >= NUM_BUCKETS)
return {};
protected:
Chunk generate() override
Block block = params->aggregator.mergeAndConvertOneBucketToBlock(*data, arena, params->final, bucket_num);
return convertToChunk(block);
}
private:
AggregatingTransformParamsPtr params;
ManyAggregatedDataVariantsPtr data;
std::shared_ptr<std::atomic<UInt32>> next_bucket_to_merge;
Arena * arena;
static constexpr UInt32 NUM_BUCKETS = 256;
};
class ConvertingAggregatedToChunksTransform : public IProcessor
{
public:
ConvertingAggregatedToChunksTransform(AggregatingTransformParamsPtr params_, ManyAggregatedDataVariantsPtr data_, size_t num_threads_)
: IProcessor({}, {params_->getHeader()})
, params(std::move(params_)), data(std::move(data_)), num_threads(num_threads_) {}
String getName() const override { return "ConvertingAggregatedToChunksTransform"; }
void work() override
{
if (data->empty())
{
auto block = stream->read();
if (!block)
return {};
auto info = std::make_shared<AggregatedChunkInfo>();
info->bucket_num = block.info.bucket_num;
info->is_overflows = block.info.is_overflows;
UInt64 num_rows = block.rows();
Chunk chunk(block.getColumns(), num_rows);
chunk.setChunkInfo(std::move(info));
return chunk;
finished = true;
return;
}
private:
/// Store params because aggregator must be destroyed after stream. Order is important.
AggregatingTransformParamsPtr params;
BlockInputStreamPtr stream;
};
}
if (!is_initialized)
{
initialize();
return;
}
if (data->at(0)->isTwoLevel())
{
/// In two-level case will only create sources.
if (inputs.empty())
createSources();
}
else
{
mergeSingleLevel();
}
}
Processors expandPipeline() override
{
for (auto & source : processors)
{
auto & out = source->getOutputs().front();
inputs.emplace_back(out.getHeader(), this);
connect(out, inputs.back());
}
return std::move(processors);
}
IProcessor::Status prepare() override
{
auto & output = outputs.front();
if (finished && !has_input)
{
output.finish();
return Status::Finished;
}
/// Check can output.
if (output.isFinished())
return Status::Finished;
if (!output.canPush())
return Status::PortFull;
if (!is_initialized)
return Status::Ready;
if (!processors.empty())
return Status::ExpandPipeline;
if (has_input)
return preparePushToOutput();
/// Single level case.
if (inputs.empty())
return Status::Ready;
/// Two-level case.
return preparePullFromInputs();
}
private:
IProcessor::Status preparePushToOutput()
{
auto & output = outputs.front();
output.push(std::move(current_chunk));
has_input = false;
if (finished)
{
output.finish();
return Status::Finished;
}
return Status::PortFull;
}
/// Read all sources and try to push current bucket.
IProcessor::Status preparePullFromInputs()
{
bool all_inputs_are_finished = true;
for (auto & input : inputs)
{
if (input.isFinished())
continue;
all_inputs_are_finished = false;
input.setNeeded();
if (input.hasData())
ready_chunks.emplace_back(input.pull());
}
moveReadyChunksToMap();
if (trySetCurrentChunkFromCurrentBucket())
return preparePushToOutput();
if (all_inputs_are_finished)
throw Exception("All sources have finished before getting enough data in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
return Status::NeedData;
}
private:
AggregatingTransformParamsPtr params;
ManyAggregatedDataVariantsPtr data;
size_t num_threads;
bool is_initialized = false;
bool has_input = false;
bool finished = false;
Chunk current_chunk;
Chunks ready_chunks;
UInt32 current_bucket_num = 0;
static constexpr Int32 NUM_BUCKETS = 256;
std::map<UInt32, Chunk> bucket_to_chunk;
Processors processors;
static Int32 getBucketFromChunk(const Chunk & chunk)
{
auto & info = chunk.getChunkInfo();
if (!info)
throw Exception("Chunk info was not set for chunk in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
auto * agg_info = typeid_cast<const AggregatedChunkInfo *>(info.get());
if (!agg_info)
throw Exception("Chunk should have AggregatedChunkInfo in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
return agg_info->bucket_num;
}
void moveReadyChunksToMap()
{
for (auto & chunk : ready_chunks)
{
auto bucket = getBucketFromChunk(chunk);
if (bucket < 0 || bucket >= NUM_BUCKETS)
throw Exception("Invalid bucket number " + toString(bucket) + " in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
if (bucket_to_chunk.count(bucket))
throw Exception("Found several chunks with the same bucket number in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
bucket_to_chunk[bucket] = std::move(chunk);
}
ready_chunks.clear();
}
void setCurrentChunk(Chunk chunk)
{
if (has_input)
throw Exception("Current chunk was already set in "
"ConvertingAggregatedToChunksTransform.", ErrorCodes::LOGICAL_ERROR);
has_input = true;
current_chunk = std::move(chunk);
}
void initialize()
{
is_initialized = true;
AggregatedDataVariantsPtr & first = data->at(0);
/// At least we need one arena in first data item per thread
if (num_threads > first->aggregates_pools.size())
{
Arenas & first_pool = first->aggregates_pools;
for (size_t j = first_pool.size(); j < num_threads; j++)
first_pool.emplace_back(std::make_shared<Arena>());
}
if (first->type == AggregatedDataVariants::Type::without_key || params->params.overflow_row)
{
params->aggregator.mergeWithoutKeyDataImpl(*data);
auto block = params->aggregator.prepareBlockAndFillWithoutKey(
*first, params->final, first->type != AggregatedDataVariants::Type::without_key);
setCurrentChunk(convertToChunk(block));
}
}
void mergeSingleLevel()
{
AggregatedDataVariantsPtr & first = data->at(0);
if (current_bucket_num > 0 || first->type == AggregatedDataVariants::Type::without_key)
{
finished = true;
return;
}
++current_bucket_num;
#define M(NAME) \
else if (first->type == AggregatedDataVariants::Type::NAME) \
params->aggregator.mergeSingleLevelDataImpl<decltype(first->NAME)::element_type>(*data);
if (false) {}
APPLY_FOR_VARIANTS_SINGLE_LEVEL(M)
#undef M
else
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
auto block = params->aggregator.prepareBlockAndFillSingleLevel(*first, params->final);
setCurrentChunk(convertToChunk(block));
finished = true;
}
void createSources()
{
AggregatedDataVariantsPtr & first = data->at(0);
auto next_bucket_to_merge = std::make_shared<std::atomic<UInt32>>(0);
for (size_t thread = 0; thread < num_threads; ++thread)
{
Arena * arena = first->aggregates_pools.at(thread).get();
auto source = std::make_shared<ConvertingAggregatedToChunksSource>(
params, data, arena, next_bucket_to_merge);
processors.emplace_back(std::move(source));
}
}
bool trySetCurrentChunkFromCurrentBucket()
{
auto it = bucket_to_chunk.find(current_bucket_num);
if (it != bucket_to_chunk.end())
{
setCurrentChunk(std::move(it->second));
++current_bucket_num;
if (current_bucket_num == NUM_BUCKETS)
finished = true;
return true;
}
return false;
}
};
AggregatingTransform::AggregatingTransform(Block header, AggregatingTransformParamsPtr params_)
: AggregatingTransform(std::move(header), std::move(params_)
@ -259,8 +546,9 @@ void AggregatingTransform::initGenerate()
if (!params->aggregator.hasTemporaryFiles())
{
auto stream = params->aggregator.mergeAndConvertToBlocks(many_data->variants, params->final, max_threads);
processors.emplace_back(std::make_shared<ConvertingAggregatedToBlocksTransform>(stream->getHeader(), params, std::move(stream)));
auto prepared_data = params->aggregator.prepareVariantsToMerge(many_data->variants);
auto prepared_data_ptr = std::make_shared<ManyAggregatedDataVariants>(std::move(prepared_data));
processors.emplace_back(std::make_shared<ConvertingAggregatedToChunksTransform>(params, std::move(prepared_data_ptr), max_threads));
}
else
{

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

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

@ -21,34 +21,26 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream(
header(header_), sync(sync_), skip_offsets(skip_offsets_),
already_written_offset_columns(already_written_offset_columns_)
{
serialization_states.reserve(header.columns());
WrittenOffsetColumns tmp_offset_columns;
IDataType::SerializeBinaryBulkSettings settings;
for (const auto & column_name : header.getNames())
{
const auto & col = header.getByName(column_name);
const auto columns = storage.getColumns();
addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets);
serialization_states.emplace_back(nullptr);
settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
}
initSkipIndices();
}
void MergedColumnOnlyOutputStream::write(const Block & block)
{
if (!initialized)
{
column_streams.clear();
serialization_states.clear();
serialization_states.reserve(header.columns());
WrittenOffsetColumns tmp_offset_columns;
IDataType::SerializeBinaryBulkSettings settings;
for (const auto & column_name : header.getNames())
{
const auto & col = block.getByName(column_name);
const auto columns = storage.getColumns();
addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets);
serialization_states.emplace_back(nullptr);
settings.getter = createStreamGetter(col.name, tmp_offset_columns, false);
col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back());
}
initSkipIndices();
initialized = true;
}
std::set<String> skip_indexes_column_names_set;
for (const auto & index : skip_indices)
std::copy(index->columns.cbegin(), index->columns.cend(),
@ -68,7 +60,6 @@ void MergedColumnOnlyOutputStream::write(const Block & block)
if (!rows)
return;
size_t new_index_offset = 0;
size_t new_current_mark = 0;
WrittenOffsetColumns offset_columns = already_written_offset_columns;
@ -106,7 +97,8 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG
serialize_settings.getter = createStreamGetter(column.name, already_written_offset_columns, skip_offsets);
column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]);
if (with_final_mark)
/// We wrote at least one row
if (with_final_mark && (index_offset != 0 || current_mark != 0))
writeFinalMark(column.name, column.type, offset_columns, skip_offsets, serialize_settings.path);
}
@ -125,7 +117,6 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG
column_streams.clear();
serialization_states.clear();
initialized = false;
return checksums;
}

View File

@ -28,7 +28,6 @@ public:
private:
Block header;
bool initialized = false;
bool sync;
bool skip_offsets;

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)
end_it = current_mutations_by_version.upper_bound(*min_version);
auto begin_it = current_mutations_by_version.begin();
size_t to_delete_count = std::distance(begin_it, end_it);
size_t done_count = std::distance(begin_it, end_it);
if (done_count <= settings->finished_mutations_to_keep)
return;
if (!truncate)
{
if (std::optional<Int64> min_version = getMinPartDataVersion())
end_it = current_mutations_by_version.upper_bound(*min_version);
size_t to_delete_count = done_count - settings->finished_mutations_to_keep;
size_t done_count = std::distance(begin_it, end_it);
if (done_count <= settings->finished_mutations_to_keep)
return;
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

@ -34,7 +34,8 @@ set the following environment variables:
### Running with runner script
The only requirement is fresh docker configured docker.
The only requirement is fresh configured docker and
docker pull yandex/clickhouse-integration-tests-runner
Notes:
* If you want to run integration tests without `sudo` you have to add your user to docker group `sudo usermod -aG docker $USER`. [More information](https://docs.docker.com/install/linux/linux-postinstall/) about docker configuration.

View File

@ -723,7 +723,8 @@ class ClickHouseInstance:
os.mkdir(config_d_dir)
os.mkdir(users_d_dir)
shutil.copy(p.join(HELPERS_DIR, 'common_instance_config.xml'), config_d_dir)
# The file is named with 0_ prefix to be processed before other configuration overloads.
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), config_d_dir)
# Generate and write macros file
macros = self.macros.copy()

View File

@ -1,4 +0,0 @@
*
!.gitignore
!source.tsv
!dictionary_preset*

View File

@ -1,411 +0,0 @@
import pytest
import os
import time
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV, assert_eq_with_retry
from generate_dictionaries import generate_structure, generate_dictionaries, DictionaryTestTable
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
cluster = None
instance = None
test_table = None
def get_status(dictionary_name):
return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
def get_last_exception(dictionary_name):
return instance.query("SELECT last_exception FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n").replace("\\'", "'")
def get_loading_start_time(dictionary_name):
s = instance.query("SELECT loading_start_time FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n")
if s == "0000-00-00 00:00:00":
return None
return time.strptime(s, "%Y-%m-%d %H:%M:%S")
def get_loading_duration(dictionary_name):
return float(instance.query("SELECT loading_duration FROM system.dictionaries WHERE name='" + dictionary_name + "'"))
def replace_in_file_in_container(file_name, what, replace_with):
instance.exec_in_container('sed -i "s/' + what + '/' + replace_with + '/g" ' + file_name)
def setup_module(module):
global cluster
global instance
global test_table
structure = generate_structure()
dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure)
cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs'))
instance = cluster.add_instance('instance', main_configs=dictionary_files)
test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv'))
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
instance.query("CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary")
test_table.create_clickhouse_source(instance)
for line in TSV(instance.query('select name from system.dictionaries')).lines:
print line,
# Create table `test.small_dict_source`
instance.query('''
drop table if exists test.small_dict_source;
create table test.small_dict_source (id UInt64, a String, b Int32, c Float64) engine=Log;
insert into test.small_dict_source values (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7);
''')
yield cluster
finally:
cluster.shutdown()
@pytest.fixture(params=[
# name, keys, use_parent
('clickhouse_hashed', ('id',), True),
('clickhouse_flat', ('id',), True),
('clickhouse_complex_integers_key_hashed', ('key0', 'key1'), False),
('clickhouse_complex_mixed_key_hashed', ('key0_str', 'key1'), False),
('clickhouse_range_hashed', ('id', 'StartDate', 'EndDate'), False),
],
ids=['clickhouse_hashed', 'clickhouse_flat',
'clickhouse_complex_integers_key_hashed',
'clickhouse_complex_mixed_key_hashed',
'clickhouse_range_hashed']
)
def dictionary_structure(started_cluster, request):
return request.param
def test_select_all(dictionary_structure):
name, keys, use_parent = dictionary_structure
query = instance.query
structure = test_table.get_structure_for_keys(keys, use_parent)
query('''
DROP TABLE IF EXISTS test.{0}
'''.format(name))
create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure)
TSV(query(create_query))
result = TSV(query('select * from test.{0}'.format(name)))
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True)
print test_table.process_diff(diff)
assert not diff
@pytest.fixture(params=[
# name, keys, use_parent
('clickhouse_cache', ('id',), True),
('clickhouse_complex_integers_key_cache', ('key0', 'key1'), False),
('clickhouse_complex_mixed_key_cache', ('key0_str', 'key1'), False)
],
ids=['clickhouse_cache', 'clickhouse_complex_integers_key_cache', 'clickhouse_complex_mixed_key_cache']
)
def cached_dictionary_structure(started_cluster, request):
return request.param
def test_select_all_from_cached(cached_dictionary_structure):
name, keys, use_parent = cached_dictionary_structure
query = instance.query
structure = test_table.get_structure_for_keys(keys, use_parent)
query('''
DROP TABLE IF EXISTS test.{0}
'''.format(name))
create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure)
TSV(query(create_query))
for i in range(4):
result = TSV(query('select * from test.{0}'.format(name)))
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=False)
print test_table.process_diff(diff)
assert not diff
key = []
for key_name in keys:
if key_name.endswith('str'):
key.append("'" + str(i) + "'")
else:
key.append(str(i))
if len(key) == 1:
key = 'toUInt64(' + str(i) + ')'
else:
key = str('(' + ','.join(key) + ')')
query("select dictGetUInt8('{0}', 'UInt8_', {1})".format(name, key))
result = TSV(query('select * from test.{0}'.format(name)))
diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True)
print test_table.process_diff(diff)
assert not diff
def test_null_value(started_cluster):
query = instance.query
assert TSV(query("select dictGetUInt8('clickhouse_cache', 'UInt8_', toUInt64(12121212))")) == TSV("1")
assert TSV(query("select dictGetString('clickhouse_cache', 'String_', toUInt64(12121212))")) == TSV("implicit-default")
assert TSV(query("select dictGetDate('clickhouse_cache', 'Date_', toUInt64(12121212))")) == TSV("2015-11-25")
# Check, that empty null_value interprets as default value
assert TSV(query("select dictGetUInt64('clickhouse_cache', 'UInt64_', toUInt64(12121212))")) == TSV("0")
assert TSV(query("select dictGetDateTime('clickhouse_cache', 'DateTime_', toUInt64(12121212))")) == TSV("0000-00-00 00:00:00")
def test_dictionary_dependency(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so these dictionary are not loaded.
assert get_status('dep_x') == 'NOT_LOADED'
assert get_status('dep_y') == 'NOT_LOADED'
assert get_status('dep_z') == 'NOT_LOADED'
# Dictionary 'dep_x' depends on 'dep_z', which depends on 'dep_y'.
# So they all should be loaded at once.
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(1))") == "air\n"
assert get_status('dep_x') == 'LOADED'
assert get_status('dep_y') == 'LOADED'
assert get_status('dep_z') == 'LOADED'
# Other dictionaries should work too.
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(1))") == "air\n"
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(1))") == "air\n"
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n"
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "YY\n"
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n"
# Update the source table.
query("insert into test.small_dict_source values (3, 'fire', 30, 8)")
# Wait for dictionaries to be reloaded.
assert_eq_with_retry(instance, "SELECT dictHas('dep_y', toUInt64(3))", "1", sleep_time = 2, retry_count = 10)
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n"
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n"
# dep_x and dep_z are updated only when there `intDiv(count(), 4)` is changed.
query("insert into test.small_dict_source values (4, 'ether', 404, 0.001)")
assert_eq_with_retry(instance, "SELECT dictHas('dep_x', toUInt64(4))", "1", sleep_time = 2, retry_count = 10)
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "fire\n"
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n"
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "fire\n"
assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "ether\n"
assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n"
assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ether\n"
def test_reload_while_loading(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so this dictionary is not loaded.
assert get_status('longload') == "NOT_LOADED"
assert get_loading_duration('longload') == 0
# It's not possible to get a value from the dictionary within 1.0 second, so the following query fails by timeout.
assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))", timeout = 1, ignore_error = True) == ""
# The dictionary is now loading.
assert get_status('longload') == "LOADING"
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert duration > 0
time.sleep(0.5) # Still loading.
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time == prev_start_time
assert duration >= prev_duration
# SYSTEM RELOAD DICTIONARY should restart loading.
query("SYSTEM RELOAD DICTIONARY 'longload'")
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time > prev_start_time
assert duration < prev_duration
time.sleep(0.5) # Still loading.
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time == prev_start_time
assert duration >= prev_duration
# SYSTEM RELOAD DICTIONARIES should restart loading again.
query("SYSTEM RELOAD DICTIONARIES")
assert get_status('longload') == "LOADING"
prev_start_time, prev_duration = start_time, duration
start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload')
assert start_time > prev_start_time
assert duration < prev_duration
# Changing the configuration file should restart loading one more time.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_longload.xml', 'sleep 100', 'sleep 0')
time.sleep(5) # Configuration files are reloaded once in 5 seconds.
# This time loading should finish quickly.
assert get_status('longload') == "LOADED"
assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))") == "6\n"
def test_reload_after_loading(started_cluster):
query = instance.query
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "8\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
# Change the dictionaries' data.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '8', '81')
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '10', '101')
# SYSTEM RELOAD 'name' reloads only the specified dictionary.
query("SYSTEM RELOAD DICTIONARY 'cmd'")
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n"
query("SYSTEM RELOAD DICTIONARY 'file'")
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "101\n"
# SYSTEM RELOAD DICTIONARIES reloads all loaded dictionaries.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '81', '82')
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '101', '102')
query("SYSTEM RELOAD DICTIONARIES")
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "82\n"
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "102\n"
# Configuration files are reloaded and lifetimes are checked automatically once in 5 seconds.
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '82', '83')
replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '102', '103')
time.sleep(5)
assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "103\n"
assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "83\n"
def test_reload_after_fail_by_system_reload(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so this dictionary is not loaded.
assert get_status("no_file") == "NOT_LOADED"
# We expect an error because the file source doesn't exist.
expected_error = "No such file"
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
assert get_status("no_file") == "FAILED"
# SYSTEM RELOAD should not change anything now, the status is still FAILED.
query("SYSTEM RELOAD DICTIONARY 'no_file'")
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))")
assert get_status("no_file") == "FAILED"
# Creating the file source makes the dictionary able to load.
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt")
query("SYSTEM RELOAD DICTIONARY 'no_file'")
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file") == "LOADED"
# Removing the file source should not spoil the loaded dictionary.
instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file.txt")
query("SYSTEM RELOAD DICTIONARY 'no_file'")
query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file") == "LOADED"
def test_reload_after_fail_by_timer(started_cluster):
query = instance.query
# dictionaries_lazy_load == false, so this dictionary is not loaded.
assert get_status("no_file_2") == "NOT_LOADED"
# We expect an error because the file source doesn't exist.
expected_error = "No such file"
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
assert get_status("no_file_2") == "FAILED"
# Passed time should not change anything now, the status is still FAILED.
time.sleep(6);
assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))")
assert get_status("no_file_2") == "FAILED"
# Creating the file source makes the dictionary able to load.
instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt")
time.sleep(6);
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file_2") == "LOADED"
# Removing the file source should not spoil the loaded dictionary.
instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt")
time.sleep(6);
query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n"
assert get_status("no_file_2") == "LOADED"
def test_reload_after_fail_in_cache_dictionary(started_cluster):
query = instance.query
query_and_get_error = instance.query_and_get_error
# Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond.
expected_error = "Table test.xypairs doesn't exist"
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))")
assert get_status("cache_xypairs") == "LOADED"
assert expected_error in get_last_exception("cache_xypairs")
# Create table `test.xypairs`.
query('''
drop table if exists test.xypairs;
create table test.xypairs (x UInt64, y UInt64) engine=Log;
insert into test.xypairs values (1, 56), (3, 78);
''')
# Cache dictionary now works.
assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "56", ignore_error=True)
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
assert get_last_exception("cache_xypairs") == ""
# Drop table `test.xypairs`.
query('drop table if exists test.xypairs')
# Values are cached so we can get them.
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56"
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
assert get_last_exception("cache_xypairs") == ""
# But we can't get a value from the source table which isn't cached.
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))")
assert expected_error in get_last_exception("cache_xypairs")
# Passed time should not spoil the cache.
time.sleep(5);
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56"
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))")
assert expected_error in get_last_exception("cache_xypairs")
# Create table `test.xypairs` again with changed values.
query('''
drop table if exists test.xypairs;
create table test.xypairs (x UInt64, y UInt64) engine=Log;
insert into test.xypairs values (1, 57), (3, 79);
''')
# The cache dictionary returns new values now.
assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "57")
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0"
query("SELECT dictGet('cache_xypairs', 'y', toUInt64(3))") == "79"
assert get_last_exception("cache_xypairs") == ""

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