Merged with master

This commit is contained in:
Nikolai Kochetov 2019-09-09 16:20:30 +03:00
commit c9cca3ba8b
181 changed files with 4585 additions and 973 deletions

View File

@ -1,16 +1,16 @@
---
name: Bug report
name: Bug report or unexpected behaviour
about: Create a report to help us improve ClickHouse
title: ''
labels: bug, issue
labels: bug
assignees: ''
---
(you don't have to strictly follow this form)
**Describe the bug**
A clear and concise description of what the bug is.
**Describe the bug or unexpected behaviour**
A clear and concise description of what works not as it is supposed to.
**How to reproduce**
* Which ClickHouse server version to use

View File

@ -0,0 +1,16 @@
---
name: Documentation issue
about: Report something incorrect or missing in documentation
title: ''
labels: documentation
assignees: BayoNet
---
(you don't have to strictly follow this form)
**Describe the issue**
A clear and concise description of what's wrong in documentation.
**Additional context**
Add any other context about the problem here.

View File

@ -0,0 +1,27 @@
---
name: Performance issue
about: Report something working slower than expected
title: ''
labels: performance
assignees: ''
---
(you don't have to strictly follow this form)
**Describe the situation**
What exactly works slower than expected?
**How to reproduce**
* Which ClickHouse server version to use
* Which interface to use, if matters
* Non-default settings, if any
* `CREATE TABLE` statements for all tables involved
* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/yandex/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary
* Queries to run that lead to slow performance
**Expected performance**
What are your performance expectation, why do you think they are realistic? Has it been working faster in older ClickHouse releases? Is it working faster in some specific other system?
**Additional context**
Add any other context about the problem here.

View File

@ -372,8 +372,8 @@ if (USE_PROTOBUF)
endif ()
if (USE_HDFS)
target_link_libraries (clickhouse_common_io PRIVATE ${HDFS3_LIBRARY})
target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${HDFS3_INCLUDE_DIR})
target_link_libraries (clickhouse_common_io PUBLIC ${HDFS3_LIBRARY})
target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR})
endif()
if (USE_BROTLI)

View File

@ -1,11 +1,11 @@
# This strings autochanged from release_lib.sh:
set(VERSION_REVISION 54425)
set(VERSION_REVISION 54426)
set(VERSION_MAJOR 19)
set(VERSION_MINOR 14)
set(VERSION_MINOR 15)
set(VERSION_PATCH 1)
set(VERSION_GITHASH adfc36917222bdb03eba069f0cad0f4f5b8f1c94)
set(VERSION_DESCRIBE v19.14.1.1-prestable)
set(VERSION_STRING 19.14.1.1)
set(VERSION_GITHASH 6f1a8c37abe6ee4e7ee74c0b5cb9c05a87417b61)
set(VERSION_DESCRIBE v19.15.1.1-prestable)
set(VERSION_STRING 19.15.1.1)
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -106,6 +106,7 @@ namespace ErrorCodes
extern const int CANNOT_SET_SIGNAL_HANDLER;
extern const int CANNOT_READLINE;
extern const int SYSTEM_ERROR;
extern const int INVALID_USAGE_OF_INPUT;
}
@ -431,8 +432,14 @@ private:
/// Load command history if present.
if (config().has("history_file"))
history_file = config().getString("history_file");
else if (!home_path.empty())
history_file = home_path + "/.clickhouse-client-history";
else
{
auto history_file_from_env = getenv("CLICKHOUSE_HISTORY_FILE");
if (history_file_from_env)
history_file = history_file_from_env;
else if (!home_path.empty())
history_file = home_path + "/.clickhouse-client-history";
}
if (!history_file.empty())
{
@ -837,9 +844,17 @@ private:
connection->forceConnected(connection_parameters.timeouts);
/// INSERT query for which data transfer is needed (not an INSERT SELECT) is processed separately.
if (insert && !insert->select)
ASTPtr input_function;
if (insert && insert->select)
insert->tryFindInputFunction(input_function);
/// INSERT query for which data transfer is needed (not an INSERT SELECT or input()) is processed separately.
if (insert && (!insert->select || input_function))
{
if (input_function && insert->format.empty())
throw Exception("FORMAT must be specified for function input()", ErrorCodes::INVALID_USAGE_OF_INPUT);
processInsertQuery();
}
else
processOrdinaryQuery();
}

View File

@ -15,6 +15,7 @@
#include <common/logger_useful.h>
#include <ext/scope_guard.h>
#include <ext/range.h>
#include <Common/SensitiveDataMasker.h>
namespace DB
{
@ -165,8 +166,7 @@ int ODBCBridge::main(const std::vector<std::string> & /*args*/)
if (config().has("query_masking_rules"))
{
context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
setLoggerSensitiveDataMasker(logger(), context->getSensitiveDataMasker());
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto server = Poco::Net::HTTPServer(

View File

@ -293,7 +293,8 @@ void MySQLHandler::comQuery(ReadBuffer & payload)
should_replace = true;
}
executeQuery(should_replace ? empty_select : payload, *out, true, connection_context, set_content_type, nullptr);
Context query_context = connection_context;
executeQuery(should_replace ? empty_select : payload, *out, true, query_context, set_content_type, nullptr);
if (!with_output)
packet_sender->sendPacket(OK_Packet(0x00, client_capability_flags, 0, 0, 0), true);

View File

@ -55,6 +55,7 @@
#include "TCPHandlerFactory.h"
#include "Common/config_version.h"
#include "MySQLHandlerFactory.h"
#include <Common/SensitiveDataMasker.h>
#if defined(__linux__)
@ -279,8 +280,6 @@ int Server::main(const std::vector<std::string> & /*args*/)
*/
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.");
@ -414,7 +413,7 @@ int Server::main(const std::vector<std::string> & /*args*/)
if (config().has("query_masking_rules"))
{
global_context->setSensitiveDataMasker(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
SensitiveDataMasker::setInstance(std::make_unique<SensitiveDataMasker>(config(), "query_masking_rules"));
}
auto main_config_reloader = std::make_unique<ConfigReloader>(config_path,
@ -426,10 +425,6 @@ 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

@ -203,6 +203,43 @@ void TCPHandler::runImpl()
state.maybe_compressed_in.reset(); /// For more accurate accounting by MemoryTracker.
});
/// Send structure of columns to client for function input()
query_context->setInputInitializer([this] (Context & context, const StoragePtr & input_storage)
{
if (&context != &query_context.value())
throw Exception("Unexpected context in Input initializer", ErrorCodes::LOGICAL_ERROR);
state.need_receive_data_for_input = true;
/// Send ColumnsDescription for input storage.
if (client_revision >= DBMS_MIN_REVISION_WITH_COLUMN_DEFAULTS_METADATA
&& query_context->getSettingsRef().input_format_defaults_for_omitted_fields)
{
sendTableColumns(input_storage->getColumns());
}
/// Send block to the client - input storage structure.
state.input_header = input_storage->getSampleBlock();
sendData(state.input_header);
});
query_context->setInputBlocksReaderCallback([&connection_settings, this] (Context & context) -> Block
{
if (&context != &query_context.value())
throw Exception("Unexpected context in InputBlocksReader", ErrorCodes::LOGICAL_ERROR);
size_t poll_interval;
int receive_timeout;
std::tie(poll_interval, receive_timeout) = getReadTimeouts(connection_settings);
if (!readDataNext(poll_interval, receive_timeout))
{
state.block_in.reset();
state.maybe_compressed_in.reset();
return Block();
}
return state.block_for_input;
});
customizeContext(*query_context);
bool may_have_embedded_data = client_revision >= DBMS_MIN_REVISION_WITH_CLIENT_SUPPORT_EMBEDDED_DATA;
@ -218,6 +255,13 @@ void TCPHandler::runImpl()
/// Does the request require receive data from client?
if (state.need_receive_data_for_insert)
processInsertQuery(connection_settings);
else if (state.need_receive_data_for_input)
{
/// It is special case for input(), all works for reading data from client will be done in callbacks.
/// state.io.in is NullAndDoCopyBlockInputStream so read it once.
state.io.in->read();
state.io.onFinish();
}
else if (state.io.pipeline.initialized())
processOrdinaryQueryWithProcessors(query_context->getSettingsRef().max_threads);
else
@ -324,7 +368,50 @@ void TCPHandler::runImpl()
}
void TCPHandler::readData(const Settings & connection_settings)
bool TCPHandler::readDataNext(const size_t & poll_interval, const int & receive_timeout)
{
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
/// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down.
while (true)
{
if (static_cast<ReadBufferFromPocoSocket &>(*in).poll(poll_interval))
break;
/// Do we need to shut down?
if (server.isCancelled())
return false;
/** Have we waited for data for too long?
* If we periodically poll, the receive_timeout of the socket itself does not work.
* Therefore, an additional check is added.
*/
double elapsed = watch.elapsedSeconds();
if (elapsed > receive_timeout)
{
std::stringstream ss;
ss << "Timeout exceeded while receiving data from client.";
ss << " Waited for " << static_cast<size_t>(elapsed) << " seconds,";
ss << " timeout is " << receive_timeout << " seconds.";
throw Exception(ss.str(), ErrorCodes::SOCKET_TIMEOUT);
}
}
/// If client disconnected.
if (in->eof())
return false;
/// We accept and process data. And if they are over, then we leave.
if (!receivePacket())
return false;
sendLogs();
return true;
}
std::tuple<size_t, int> TCPHandler::getReadTimeouts(const Settings & connection_settings)
{
const auto receive_timeout = query_context->getSettingsRef().receive_timeout.value;
@ -334,48 +421,21 @@ void TCPHandler::readData(const Settings & connection_settings)
constexpr size_t min_poll_interval = 5000; // 5 ms
size_t poll_interval = std::max(min_poll_interval, std::min(default_poll_interval, current_poll_interval));
return std::make_tuple(poll_interval, receive_timeout.totalSeconds());
}
void TCPHandler::readData(const Settings & connection_settings)
{
size_t poll_interval;
int receive_timeout;
std::tie(poll_interval, receive_timeout) = getReadTimeouts(connection_settings);
sendLogs();
while (true)
{
Stopwatch watch(CLOCK_MONOTONIC_COARSE);
/// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down.
while (true)
{
if (static_cast<ReadBufferFromPocoSocket &>(*in).poll(poll_interval))
break;
/// Do we need to shut down?
if (server.isCancelled())
return;
/** Have we waited for data for too long?
* If we periodically poll, the receive_timeout of the socket itself does not work.
* Therefore, an additional check is added.
*/
double elapsed = watch.elapsedSeconds();
if (elapsed > receive_timeout.totalSeconds())
{
std::stringstream ss;
ss << "Timeout exceeded while receiving data from client.";
ss << " Waited for " << static_cast<size_t>(elapsed) << " seconds,";
ss << " timeout is " << receive_timeout.totalSeconds() << " seconds.";
throw Exception(ss.str(), ErrorCodes::SOCKET_TIMEOUT);
}
}
/// If client disconnected.
if (in->eof())
if (!readDataNext(poll_interval, receive_timeout))
return;
/// We accept and process data. And if they are over, then we leave.
if (!receivePacket())
break;
sendLogs();
}
}
@ -904,7 +964,7 @@ bool TCPHandler::receiveData()
{
/// If there is an insert request, then the data should be written directly to `state.io.out`.
/// Otherwise, we write the blocks in the temporary `external_table_name` table.
if (!state.need_receive_data_for_insert)
if (!state.need_receive_data_for_insert && !state.need_receive_data_for_input)
{
StoragePtr storage;
/// If such a table does not exist, create it.
@ -918,7 +978,9 @@ bool TCPHandler::receiveData()
/// The data will be written directly to the table.
state.io.out = storage->write(ASTPtr(), *query_context);
}
if (block)
if (state.need_receive_data_for_input)
state.block_for_input = block;
else
state.io.out->write(block);
return true;
}
@ -960,6 +1022,8 @@ void TCPHandler::initBlockInput()
Block header;
if (state.io.out)
header = state.io.out->getHeader();
else if (state.need_receive_data_for_input)
header = state.input_header;
last_block_in.header = header;
last_block_in.compression = state.compression;

View File

@ -64,6 +64,13 @@ struct QueryState
/// Request requires data from the client (INSERT, but not INSERT SELECT).
bool need_receive_data_for_insert = false;
/// Request requires data from client for function input()
bool need_receive_data_for_input = false;
/// temporary place for incoming data block for input()
Block block_for_input;
/// sample block from StorageInput
Block input_header;
/// To output progress, the difference after the previous sending of progress.
Progress progress;
@ -147,7 +154,9 @@ private:
bool receivePacket();
void receiveQuery();
bool receiveData();
bool readDataNext(const size_t & poll_interval, const int & receive_timeout);
void readData(const Settings & global_settings);
std::tuple<size_t, int> getReadTimeouts(const Settings & global_settings);
[[noreturn]] void receiveUnexpectedData();
[[noreturn]] void receiveUnexpectedQuery();

View File

@ -1,3 +1,5 @@
<?xml version="1.0"?>
<!-- Config for test server -->
<yandex>
<query_masking_rules>
<!-- An example of query masking rule to remove profanity.
@ -8,5 +10,10 @@
<regexp>(?i:shit)</regexp>
<replace>substance</replace>
</rule>
<!-- This rule is used in tests. -->
<rule>
<regexp>TOPSECRET.TOPSECRET</regexp>
<replace>[hidden]</replace>
</rule>
</query_masking_rules>
</yandex>

View File

@ -447,7 +447,7 @@
<query_masking_rules>
<rule>
<name>hide SSN</name>
<regexp>(^|\D)\d{3}-\d{2}-\d{4}($|\D)</regexp>
<regexp>\b\d{3}-\d{2}-\d{4}\b</regexp>
<replace>000-00-0000</replace>
</rule>
</query_masking_rules>

View File

@ -10,6 +10,7 @@
#include <Columns/ColumnArray.h>
#include <Common/HashTable/HashSet.h>
#include <Common/HashTable/HashTableKeyHolder.h>
#include <Common/assert_cast.h>
#include <AggregateFunctions/IAggregateFunction.h>
@ -132,11 +133,6 @@ struct AggregateFunctionGroupUniqArrayGenericData
Set value;
};
/// Helper function for deserialize and insert for the class AggregateFunctionGroupUniqArrayGeneric
template <bool is_plain_column>
static StringRef getSerializationImpl(const IColumn & column, size_t row_num, Arena & arena);
template <bool is_plain_column>
static void deserializeAndInsertImpl(StringRef str, IColumn & data_to);
@ -154,9 +150,18 @@ class AggregateFunctionGroupUniqArrayGeneric
using State = AggregateFunctionGroupUniqArrayGenericData;
static StringRef getSerialization(const IColumn & column, size_t row_num, Arena & arena)
static auto getKeyHolder(const IColumn & column, size_t row_num, Arena & arena)
{
return getSerializationImpl<is_plain_column>(column, row_num, arena);
if constexpr (is_plain_column)
{
return ArenaKeyHolder{column.getDataAt(row_num), arena};
}
else
{
const char * begin = nullptr;
StringRef serialized = column.serializeValueIntoArena(row_num, arena, begin);
return SerializedKeyHolder{serialized, arena};
}
}
static void deserializeAndInsert(StringRef str, IColumn & data_to)
@ -209,26 +214,13 @@ public:
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
auto & set = this->data(place).value;
if (limit_num_elems && set.size() >= max_elems)
return;
bool inserted;
State::Set::iterator it;
if (limit_num_elems && set.size() >= max_elems)
return;
StringRef str_serialized = getSerialization(*columns[0], row_num, *arena);
set.emplace(str_serialized, it, inserted);
if constexpr (!is_plain_column)
{
if (!inserted)
arena->rollback(str_serialized.size);
}
else
{
if (inserted)
it->getValueMutable().data = arena->insert(str_serialized.data, str_serialized.size);
}
auto key_holder = getKeyHolder(*columns[0], row_num, *arena);
set.emplace(key_holder, it, inserted);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
@ -241,15 +233,11 @@ public:
for (auto & rhs_elem : rhs_set)
{
if (limit_num_elems && cur_set.size() >= max_elems)
return ;
cur_set.emplace(rhs_elem.getValue(), it, inserted);
if (inserted)
{
if (it->getValue().size)
it->getValueMutable().data = arena->insert(it->getValue().data, it->getValue().size);
else
it->getValueMutable().data = nullptr;
}
return;
// We have to copy the keys to our arena.
assert(arena != nullptr);
cur_set.emplace(ArenaKeyHolder{rhs_elem.getValue(), *arena}, it, inserted);
}
}
@ -271,20 +259,6 @@ public:
const char * getHeaderFilePath() const override { return __FILE__; }
};
template <>
inline StringRef getSerializationImpl<false>(const IColumn & column, size_t row_num, Arena & arena)
{
const char * begin = nullptr;
return column.serializeValueIntoArena(row_num, arena, begin);
}
template <>
inline StringRef getSerializationImpl<true>(const IColumn & column, size_t row_num, Arena &)
{
return column.getDataAt(row_num);
}
template <>
inline void deserializeAndInsertImpl<false>(StringRef str, IColumn & data_to)
{

View File

@ -31,8 +31,9 @@ namespace ErrorCodes
ConnectionPoolWithFailover::ConnectionPoolWithFailover(
ConnectionPoolPtrs nested_pools_,
LoadBalancing load_balancing,
time_t decrease_error_period_)
: Base(std::move(nested_pools_), decrease_error_period_, &Logger::get("ConnectionPoolWithFailover"))
time_t decrease_error_period_,
size_t max_error_cap_)
: Base(std::move(nested_pools_), decrease_error_period_, max_error_cap_, &Logger::get("ConnectionPoolWithFailover"))
, default_load_balancing(load_balancing)
{
const std::string & local_hostname = getFQDNOrHostName();
@ -73,6 +74,31 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts
return Base::get(try_get_entry, get_priority);
}
ConnectionPoolWithFailover::Status ConnectionPoolWithFailover::getStatus() const
{
const Base::PoolStates states = getPoolStates();
const Base::NestedPools pools = nested_pools;
assert(states.size() == pools.size());
ConnectionPoolWithFailover::Status result;
result.reserve(states.size());
const time_t since_last_error_decrease = time(nullptr) - last_error_decrease_time;
for (size_t i = 0; i < states.size(); ++i)
{
const auto rounds_to_zero_errors = states[i].error_count ? bitScanReverse(states[i].error_count) + 1 : 0;
const auto seconds_to_zero_errors = std::max(static_cast<time_t>(0), rounds_to_zero_errors * decrease_error_period - since_last_error_decrease);
result.emplace_back(NestedPoolStatus{
pools[i].get(),
states[i].error_count,
std::chrono::seconds{seconds_to_zero_errors}
});
}
return result;
}
std::vector<IConnectionPool::Entry> ConnectionPoolWithFailover::getMany(const ConnectionTimeouts & timeouts,
const Settings * settings,
PoolMode pool_mode)

View File

@ -3,6 +3,9 @@
#include <Common/PoolWithFailoverBase.h>
#include <Client/ConnectionPool.h>
#include <chrono>
#include <vector>
namespace DB
{
@ -34,7 +37,8 @@ public:
ConnectionPoolWithFailover(
ConnectionPoolPtrs nested_pools_,
LoadBalancing load_balancing,
time_t decrease_error_period_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD);
time_t decrease_error_period_ = DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD,
size_t max_error_cap = DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT);
using Entry = IConnectionPool::Entry;
@ -64,6 +68,16 @@ public:
PoolMode pool_mode,
const QualifiedTableName & table_to_check);
struct NestedPoolStatus
{
const IConnectionPool * pool;
size_t error_count;
std::chrono::seconds estimated_recovery_time;
};
using Status = std::vector<NestedPoolStatus>;
Status getStatus() const;
private:
/// Get the values of relevant settings and call Base::getMany()
std::vector<TryResult> getManyImpl(

View File

@ -170,11 +170,14 @@ public:
/** Rollback just performed allocation.
* Must pass size not more that was just allocated.
* Return the resulting head pointer, so that the caller can assert that
* the allocation it intended to roll back was indeed the last one.
*/
void rollback(size_t size)
void * rollback(size_t size)
{
head->pos -= size;
ASAN_POISON_MEMORY_REGION(head->pos, size + pad_right);
return head->pos;
}
/** Begin or expand allocation of contiguous piece of memory without alignment.

View File

@ -1,6 +1,8 @@
#pragma once
#include <Common/HashTable/HashTable.h>
#include <Common/HashTable/HashTableKeyHolder.h>
#include <Common/ColumnsHashingImpl.h>
#include <Common/Arena.h>
#include <Common/LRUCache.h>
@ -57,7 +59,7 @@ struct HashMethodOneNumber
using Base::getHash; /// (const Data & data, size_t row, Arena & pool) -> size_t
/// Is used for default implementation in HashMethodBase.
FieldType getKey(size_t row, Arena &) const { return unalignedLoad<FieldType>(vec + row * sizeof(FieldType)); }
FieldType getKeyHolder(size_t row, Arena &) const { return unalignedLoad<FieldType>(vec + row * sizeof(FieldType)); }
/// Get StringRef from value which can be inserted into column.
static StringRef getValueRef(const Value & value)
@ -86,24 +88,24 @@ struct HashMethodString
chars = column_string.getChars().data();
}
auto getKey(ssize_t row, Arena &) const
auto getKeyHolder(ssize_t row, [[maybe_unused]] Arena & pool) const
{
return StringRef(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1);
StringRef key(chars + offsets[row - 1], offsets[row] - offsets[row - 1] - 1);
if constexpr (place_string_to_arena)
{
return ArenaKeyHolder{key, pool};
}
else
{
return key;
}
}
static StringRef getValueRef(const Value & value) { return value.first; }
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
static ALWAYS_INLINE void onNewKey([[maybe_unused]] StringRef & key, [[maybe_unused]] Arena & pool)
{
if constexpr (place_string_to_arena)
{
if (key.size)
key.data = pool.insert(key.data, key.size);
}
}
};
@ -126,17 +128,24 @@ struct HashMethodFixedString
chars = &column_string.getChars();
}
StringRef getKey(size_t row, Arena &) const { return StringRef(&(*chars)[row * n], n); }
auto getKeyHolder(size_t row, [[maybe_unused]] Arena & pool) const
{
StringRef key(&(*chars)[row * n], n);
if constexpr (place_string_to_arena)
{
return ArenaKeyHolder{key, pool};
}
else
{
return key;
}
}
static StringRef getValueRef(const Value & value) { return value.first; }
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, use_cache>;
static ALWAYS_INLINE void onNewKey([[maybe_unused]] StringRef & key, [[maybe_unused]] Arena & pool)
{
if constexpr (place_string_to_arena)
key.data = pool.insert(key.data, key.size);
}
};
@ -316,10 +325,10 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
}
}
/// Get the key from the key columns for insertion into the hash table.
ALWAYS_INLINE auto getKey(size_t row, Arena & pool) const
/// Get the key holder from the key columns for insertion into the hash table.
ALWAYS_INLINE auto getKeyHolder(size_t row, Arena & pool) const
{
return Base::getKey(getIndexAt(row), pool);
return Base::getKeyHolder(getIndexAt(row), pool);
}
template <typename Data>
@ -347,30 +356,23 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
return EmplaceResult(false);
}
auto key = getKey(row_, pool);
auto key_holder = getKeyHolder(row_, pool);
bool inserted = false;
typename Data::iterator it;
if (saved_hash)
data.emplace(key, it, inserted, saved_hash[row]);
data.emplace(key_holder, it, inserted, saved_hash[row]);
else
data.emplace(key, it, inserted);
data.emplace(key_holder, it, inserted);
visit_cache[row] = VisitValue::Found;
if (inserted)
{
if constexpr (has_mapped)
{
new(&it->getSecond()) Mapped();
Base::onNewKey(it->getFirstMutable(), pool);
}
else
Base::onNewKey(*it, pool);
}
if constexpr (has_mapped)
{
if (inserted)
{
new (&it->getSecond()) Mapped();
}
mapped_cache[row] = it->getSecond();
return EmplaceResult(it->getSecond(), mapped_cache[row], inserted);
}
@ -407,13 +409,13 @@ struct HashMethodSingleLowCardinalityColumn : public SingleColumnMethod
return FindResult(visit_cache[row] == VisitValue::Found);
}
auto key = getKey(row_, pool);
auto key_holder = getKeyHolder(row_, pool);
typename Data::iterator it;
if (saved_hash)
it = data.find(key, saved_hash[row]);
it = data.find(*key_holder, saved_hash[row]);
else
it = data.find(key);
it = data.find(*key_holder);
bool found = it != data.end();
visit_cache[row] = found ? VisitValue::Found : VisitValue::NotFound;
@ -493,7 +495,7 @@ struct HashMethodKeysFixed
}
}
ALWAYS_INLINE Key getKey(size_t row, Arena &) const
ALWAYS_INLINE Key getKeyHolder(size_t row, Arena &) const
{
if constexpr (has_nullable_keys)
{
@ -532,12 +534,12 @@ struct HashMethodSerialized
protected:
friend class columns_hashing_impl::HashMethodBase<Self, Value, Mapped, false>;
ALWAYS_INLINE StringRef getKey(size_t row, Arena & pool) const
ALWAYS_INLINE SerializedKeyHolder getKeyHolder(size_t row, Arena & pool) const
{
return serializeKeysToPoolContiguous(row, keys_size, key_columns, pool);
return SerializedKeyHolder{
serializeKeysToPoolContiguous(row, keys_size, key_columns, pool),
pool};
}
static ALWAYS_INLINE void onExistingKey(StringRef & key, Arena & pool) { pool.rollback(key.size); }
};
/// For the case when there is one string key.
@ -554,7 +556,10 @@ struct HashMethodHashed
HashMethodHashed(ColumnRawPtrs key_columns_, const Sizes &, const HashMethodContextPtr &)
: key_columns(std::move(key_columns_)) {}
ALWAYS_INLINE Key getKey(size_t row, Arena &) const { return hash128(row, key_columns.size(), key_columns); }
ALWAYS_INLINE Key getKeyHolder(size_t row, Arena &) const
{
return hash128(row, key_columns.size(), key_columns);
}
static ALWAYS_INLINE StringRef getValueRef(const Value & value)
{

View File

@ -2,6 +2,7 @@
#include <Columns/IColumn.h>
#include <Common/assert_cast.h>
#include <Common/HashTable/HashTableKeyHolder.h>
#include <Interpreters/AggregationCommon.h>
@ -117,26 +118,22 @@ public:
template <typename Data>
ALWAYS_INLINE EmplaceResult emplaceKey(Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
return emplaceKeyImpl(key, data, pool);
auto key_holder = static_cast<Derived &>(*this).getKeyHolder(row, pool);
return emplaceImpl(key_holder, data);
}
template <typename Data>
ALWAYS_INLINE FindResult findKey(Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
auto res = findKeyImpl(key, data);
static_cast<Derived &>(*this).onExistingKey(key, pool);
return res;
auto key_holder = static_cast<Derived &>(*this).getKeyHolder(row, pool);
return findKeyImpl(keyHolderGetKey(key_holder), data);
}
template <typename Data>
ALWAYS_INLINE size_t getHash(const Data & data, size_t row, Arena & pool)
{
auto key = static_cast<Derived &>(*this).getKey(row, pool);
auto res = data.hash(key);
static_cast<Derived &>(*this).onExistingKey(key, pool);
return res;
auto key_holder = static_cast<Derived &>(*this).getKeyHolder(row, pool);
return data.hash(keyHolderGetKey(key_holder));
}
protected:
@ -157,20 +154,13 @@ protected:
}
}
template <typename Key>
static ALWAYS_INLINE void onNewKey(Key & /*key*/, Arena & /*pool*/) {}
template <typename Key>
static ALWAYS_INLINE void onExistingKey(Key & /*key*/, Arena & /*pool*/) {}
template <typename Data, typename Key>
ALWAYS_INLINE EmplaceResult emplaceKeyImpl(Key key, Data & data, Arena & pool)
template <typename Data, typename KeyHolder>
ALWAYS_INLINE EmplaceResult emplaceImpl(KeyHolder & key_holder, Data & data)
{
if constexpr (Cache::consecutive_keys_optimization)
{
if (cache.found && cache.check(key))
if (cache.found && cache.check(keyHolderGetKey(key_holder)))
{
static_cast<Derived &>(*this).onExistingKey(key, pool);
if constexpr (has_mapped)
return EmplaceResult(cache.value.second, cache.value.second, false);
else
@ -180,7 +170,7 @@ protected:
typename Data::iterator it;
bool inserted = false;
data.emplace(key, it, inserted);
data.emplace(key_holder, it, inserted);
[[maybe_unused]] Mapped * cached = nullptr;
if constexpr (has_mapped)
@ -191,13 +181,8 @@ protected:
if constexpr (has_mapped)
{
new(&it->getSecond()) Mapped();
static_cast<Derived &>(*this).onNewKey(it->getFirstMutable(), pool);
}
else
static_cast<Derived &>(*this).onNewKey(it->getValueMutable(), pool);
}
else
static_cast<Derived &>(*this).onExistingKey(key, pool);
if constexpr (consecutive_keys_optimization)
{

View File

@ -449,6 +449,9 @@ namespace ErrorCodes
extern const int READONLY_SETTING = 472;
extern const int DEADLOCK_AVOIDED = 473;
extern const int INVALID_TEMPLATE_FORMAT = 474;
extern const int INVALID_WITH_FILL_EXPRESSION = 475;
extern const int WITH_TIES_WITHOUT_ORDER_BY = 476;
extern const int INVALID_USAGE_OF_INPUT = 477;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -262,8 +262,9 @@ public:
iterator end() { return iterator(this, buf + BUFFER_SIZE); }
protected:
void ALWAYS_INLINE emplaceImpl(Key x, iterator & it, bool & inserted)
public:
/// The last parameter is unused but exists for compatibility with HashTable interface.
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t /* hash */ = 0)
{
it = iterator(this, &buf[x]);
@ -278,22 +279,16 @@ protected:
++m_size;
}
public:
std::pair<iterator, bool> ALWAYS_INLINE insert(const value_type & x)
{
std::pair<iterator, bool> res;
emplaceImpl(Cell::getKey(x), res.first, res.second);
emplace(Cell::getKey(x), res.first, res.second);
if (res.second)
res.first.ptr->setMapped(x);
return res;
}
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted) { emplaceImpl(x, it, inserted); }
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t) { emplaceImpl(x, it, inserted); }
iterator ALWAYS_INLINE find(Key x)
{
return !buf[x].isZero(*this) ? iterator(this, &buf[x]) : end();

View File

@ -21,6 +21,7 @@
#include <IO/VarInt.h>
#include <Common/HashTable/HashTableAllocator.h>
#include <Common/HashTable/HashTableKeyHolder.h>
#ifdef DBMS_HASH_MAP_DEBUG_RESIZES
#include <iostream>
@ -630,6 +631,8 @@ protected:
/// If the key is zero, insert it into a special place and return true.
/// We don't have to persist a zero key, because it's not actually inserted.
/// That's why we just take a Key by value, an not a key holder.
bool ALWAYS_INLINE emplaceIfZero(Key x, iterator & it, bool & inserted, size_t hash_value)
{
/// If it is claimed that the zero key can not be inserted into the table.
@ -655,17 +658,23 @@ protected:
return false;
}
void ALWAYS_INLINE emplaceNonZeroImpl(size_t place_value, Key x, iterator & it, bool & inserted, size_t hash_value)
template <typename KeyHolder>
void ALWAYS_INLINE emplaceNonZeroImpl(size_t place_value, KeyHolder && key_holder,
iterator & it, bool & inserted, size_t hash_value)
{
it = iterator(this, &buf[place_value]);
if (!buf[place_value].isZero(*this))
{
keyHolderDiscardKey(key_holder);
inserted = false;
return;
}
new(&buf[place_value]) Cell(x, *this);
keyHolderPersistKey(key_holder);
const auto & key = keyHolderGetKey(key_holder);
new(&buf[place_value]) Cell(key, *this);
buf[place_value].setHash(hash_value);
inserted = true;
++m_size;
@ -687,19 +696,21 @@ protected:
throw;
}
it = find(x, hash_value);
it = find(keyHolderGetKey(key_holder), hash_value);
}
}
/// Only for non-zero keys. Find the right place, insert the key there, if it does not already exist. Set iterator to the cell in output parameter.
void ALWAYS_INLINE emplaceNonZero(Key x, iterator & it, bool & inserted, size_t hash_value)
template <typename KeyHolder>
void ALWAYS_INLINE emplaceNonZero(KeyHolder && key_holder, iterator & it,
bool & inserted, size_t hash_value)
{
size_t place_value = findCell(x, hash_value, grower.place(hash_value));
emplaceNonZeroImpl(place_value, x, it, inserted, hash_value);
const auto & key = keyHolderGetKey(key_holder);
size_t place_value = findCell(key, hash_value, grower.place(hash_value));
emplaceNonZeroImpl(place_value, key_holder, it, inserted, hash_value);
}
public:
/// Insert a value. In the case of any more complex values, it is better to use the `emplace` function.
std::pair<iterator, bool> ALWAYS_INLINE insert(const value_type & x)
@ -708,7 +719,9 @@ public:
size_t hash_value = hash(Cell::getKey(x));
if (!emplaceIfZero(Cell::getKey(x), res.first, res.second, hash_value))
{
emplaceNonZero(Cell::getKey(x), res.first, res.second, hash_value);
}
if (res.second)
res.first.ptr->setMapped(x);
@ -739,19 +752,20 @@ public:
* if (inserted)
* new(&it->second) Mapped(value);
*/
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted)
template <typename KeyHolder>
void ALWAYS_INLINE emplace(KeyHolder && key_holder, iterator & it, bool & inserted)
{
size_t hash_value = hash(x);
if (!emplaceIfZero(x, it, inserted, hash_value))
emplaceNonZero(x, it, inserted, hash_value);
const auto & key = keyHolderGetKey(key_holder);
emplace(key_holder, it, inserted, hash(key));
}
/// Same, but with a precalculated value of hash function.
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t hash_value)
template <typename KeyHolder>
void ALWAYS_INLINE emplace(KeyHolder && key_holder, iterator & it,
bool & inserted, size_t hash_value)
{
if (!emplaceIfZero(x, it, inserted, hash_value))
emplaceNonZero(x, it, inserted, hash_value);
const auto & key = keyHolderGetKey(key_holder);
if (!emplaceIfZero(key, it, inserted, hash_value))
emplaceNonZero(key_holder, it, inserted, hash_value);
}
/// Copy the cell from another hash table. It is assumed that the cell is not zero, and also that there was no such key in the table yet.

View File

@ -0,0 +1,130 @@
#pragma once
#include <Common/Arena.h>
/**
* In some aggregation scenarios, when adding a key to the hash table, we
* start with a temporary key object, and if it turns out to be a new key,
* we must make it persistent (e.g. copy to an Arena) and use the resulting
* persistent object as hash table key. This happens only for StringRef keys,
* because other key types are stored by value, but StringRef is a pointer-like
* type: the actual data are stored elsewhere. Even for StringRef, we don't
* make a persistent copy of the key in each of the following cases:
* 1) the aggregation method doesn't use temporary keys, so they're persistent
* from the start;
* 1) the key is already present in the hash table;
* 3) that particular key is stored by value, e.g. a short StringRef key in
* StringHashMap.
*
* In the past, the caller was responsible for making the key persistent after
* in was inserted. emplace() returned whether the key is new or not, so the
* caller only stored new keys (this is case (2) from the above list). However,
* now we are adding a compound hash table for StringRef keys, so case (3)
* appears. The decision about persistence now depends on some properties of
* the key, and the logic of this decision is tied to the particular hash table
* implementation. This means that the hash table user now doesn't have enough
* data and logic to make this decision by itself.
*
* To support these new requirements, we now manage key persistence by passing
* a special key holder to emplace(), which has the functions to make the key
* persistent or to discard it. emplace() then calls these functions at the
* appropriate moments.
*
* This approach has the following benefits:
* - no extra runtime branches in the caller to make the key persistent.
* - no additional data is stored in the hash table itself, which is important
* when it's used in aggregate function states.
* - no overhead when the key memory management isn't needed: we just pass the
* bare key without any wrapper to emplace(), and the default callbacks do
* nothing.
*
* This file defines the default key persistence functions, as well as two
* different key holders and corresponding functions for storing StringRef
* keys to Arena.
*/
/**
* Returns the key. Can return the temporary key initially.
* After the call to keyHolderPersistKey(), must return the persistent key.
*/
template <typename Key>
inline Key & ALWAYS_INLINE keyHolderGetKey(Key && key) { return key; }
/**
* Make the key persistent. keyHolderGetKey() must return the persistent key
* after this call.
*/
template <typename Key>
inline void ALWAYS_INLINE keyHolderPersistKey(Key &&) {}
/**
* Discard the key. Calling keyHolderGetKey() is ill-defined after this.
*/
template <typename Key>
inline void ALWAYS_INLINE keyHolderDiscardKey(Key &&) {}
namespace DB
{
/**
* ArenaKeyHolder is a key holder for hash tables that serializes a StringRef
* key to an Arena.
*/
struct ArenaKeyHolder
{
StringRef key;
Arena & pool;
};
}
inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::ArenaKeyHolder & holder)
{
return holder.key;
}
inline void ALWAYS_INLINE keyHolderPersistKey(DB::ArenaKeyHolder & holder)
{
// Hash table shouldn't ask us to persist a zero key
assert(holder.key.size > 0);
holder.key.data = holder.pool.insert(holder.key.data, holder.key.size);
}
inline void ALWAYS_INLINE keyHolderDiscardKey(DB::ArenaKeyHolder &)
{
}
namespace DB
{
/**
* SerializedKeyHolder is a key holder for a StringRef key that is already
* serialized to an Arena. The key must be the last allocation in this Arena,
* and is discarded by rolling back the allocation.
*/
struct SerializedKeyHolder
{
StringRef key;
Arena & pool;
};
}
inline StringRef & ALWAYS_INLINE keyHolderGetKey(DB::SerializedKeyHolder & holder)
{
return holder.key;
}
inline void ALWAYS_INLINE keyHolderPersistKey(DB::SerializedKeyHolder &)
{
}
inline void ALWAYS_INLINE keyHolderDiscardKey(DB::SerializedKeyHolder & holder)
{
[[maybe_unused]] void * new_head = holder.pool.rollback(holder.key.size);
assert(new_head == holder.key.data);
holder.key.data = nullptr;
holder.key.size = 0;
}

View File

@ -235,19 +235,22 @@ public:
* if (inserted)
* new(&it->second) Mapped(value);
*/
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted)
template <typename KeyHolder>
void ALWAYS_INLINE emplace(KeyHolder && key_holder, iterator & it, bool & inserted)
{
size_t hash_value = hash(x);
emplace(x, it, inserted, hash_value);
size_t hash_value = hash(keyHolderGetKey(key_holder));
emplace(key_holder, it, inserted, hash_value);
}
/// Same, but with a precalculated values of hash function.
void ALWAYS_INLINE emplace(Key x, iterator & it, bool & inserted, size_t hash_value)
template <typename KeyHolder>
void ALWAYS_INLINE emplace(KeyHolder && key_holder, iterator & it,
bool & inserted, size_t hash_value)
{
size_t buck = getBucketFromHash(hash_value);
typename Impl::iterator impl_it;
impls[buck].emplace(x, impl_it, inserted, hash_value);
impls[buck].emplace(key_holder, impl_it, inserted, hash_value);
it = iterator(this, buck, impl_it);
}

View File

@ -56,9 +56,11 @@ public:
PoolWithFailoverBase(
NestedPools nested_pools_,
time_t decrease_error_period_,
size_t max_error_cap_,
Logger * log_)
: nested_pools(std::move(nested_pools_))
, decrease_error_period(decrease_error_period_)
, max_error_cap(max_error_cap_)
, shared_pool_states(nested_pools.size())
, log(log_)
{
@ -120,12 +122,14 @@ protected:
/// This function returns a copy of pool states to avoid race conditions when modifying shared pool states.
PoolStates updatePoolStates();
PoolStates getPoolStates() const;
NestedPools nested_pools;
const time_t decrease_error_period;
const size_t max_error_cap;
std::mutex pool_states_mutex;
mutable std::mutex pool_states_mutex;
PoolStates shared_pool_states;
/// The time when error counts were last decreased.
time_t last_error_decrease_time = 0;
@ -193,7 +197,10 @@ PoolWithFailoverBase<TNestedPool>::getMany(
{
std::lock_guard lock(pool_states_mutex);
for (const ShuffledPool & pool: shuffled_pools)
shared_pool_states[pool.index].error_count += pool.error_count;
{
auto & pool_state = shared_pool_states[pool.index];
pool_state.error_count = std::min(max_error_cap, pool_state.error_count + pool.error_count);
}
});
std::string fail_messages;
@ -236,7 +243,7 @@ PoolWithFailoverBase<TNestedPool>::getMany(
<< (shuffled_pool.error_count + 1) << ", reason: " << fail_message);
ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry);
++shuffled_pool.error_count;
shuffled_pool.error_count = std::min(max_error_cap, shuffled_pool.error_count + 1);
if (shuffled_pool.error_count >= max_tries)
{
@ -297,7 +304,8 @@ void PoolWithFailoverBase<TNestedPool>::reportError(const Entry & entry)
if (nested_pools[i]->contains(entry))
{
std::lock_guard lock(pool_states_mutex);
++shared_pool_states[i].error_count;
auto & pool_state = shared_pool_states[i];
pool_state.error_count = std::min(max_error_cap, pool_state.error_count + 1);
return;
}
}
@ -373,3 +381,11 @@ PoolWithFailoverBase<TNestedPool>::updatePoolStates()
}
return result;
}
template <typename TNestedPool>
typename PoolWithFailoverBase<TNestedPool>::PoolStates
PoolWithFailoverBase<TNestedPool>::getPoolStates() const
{
std::lock_guard lock(pool_states_mutex);
return shared_pool_states;
}

View File

@ -24,6 +24,7 @@ namespace DB
namespace ErrorCodes
{
extern const int CANNOT_COMPILE_REGEXP;
extern const int LOGICAL_ERROR;
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int INVALID_CONFIG_PARAMETER;
}
@ -38,7 +39,9 @@ private:
const RE2 regexp;
const re2::StringPiece replacement;
#ifndef NDEBUG
mutable std::atomic<std::uint64_t> matches_count = 0;
#endif
public:
//* TODO: option with hyperscan? https://software.intel.com/en-us/articles/why-and-how-to-replace-pcre-with-hyperscan
@ -61,15 +64,37 @@ public:
uint64_t apply(std::string & data) const
{
auto m = RE2::GlobalReplace(&data, regexp, replacement);
#ifndef NDEBUG
matches_count += m;
#endif
return m;
}
const std::string & getName() const { return name; }
const std::string & getReplacementString() const { return replacement_string; }
#ifndef NDEBUG
uint64_t getMatchesCount() const { return matches_count; }
#endif
};
std::unique_ptr<SensitiveDataMasker> SensitiveDataMasker::sensitive_data_masker = nullptr;
void SensitiveDataMasker::setInstance(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)
{
sensitive_data_masker = std::move(sensitive_data_masker_);
}
}
SensitiveDataMasker * SensitiveDataMasker::getInstance()
{
return sensitive_data_masker.get();
}
SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
@ -90,7 +115,7 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration
if (!used_names.insert(rule_name).second)
{
throw Exception(
"Query_masking_rules configuration contains more than one rule named '" + rule_name + "'.",
"query_masking_rules configuration contains more than one rule named '" + rule_name + "'.",
ErrorCodes::INVALID_CONFIG_PARAMETER);
}
@ -99,7 +124,7 @@ SensitiveDataMasker::SensitiveDataMasker(const Poco::Util::AbstractConfiguration
if (regexp.empty())
{
throw Exception(
"Query_masking_rules configuration, rule '" + rule_name + "' has no <regexp> node or <regexp> is empty.",
"query_masking_rules configuration, rule '" + rule_name + "' has no <regexp> node or <regexp> is empty.",
ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}

View File

@ -4,7 +4,6 @@
#include <vector>
#include <cstdint>
namespace Poco
{
namespace Util
@ -13,6 +12,32 @@ namespace Util
}
}
/// SensitiveDataMasker allows to remove sensitive data from queries using set of regexp-based rules
/// It's used as a singelton via getInstance method
/// Initially it's empty (nullptr) and after manual initialization
/// (one-time, done by setInstance call) it takes the proper value which
/// is stored in unique_ptr.
/// It looks like the singelton is the best option here, as
/// two users of that object (OwnSplitChannel & Interpreters/executeQuery)
/// can't own/share that Masker properly without syncronization & locks,
/// and we can't afford setting global locks for each logged line.
/// I've considered singleton alternatives, but it's unclear who should own the object,
/// and it introduce unnecessary complexity in implementation (passing references back and forward):
///
/// context can't own, as Context is destroyed before logger,
/// and logger lives longer and logging can still happen after Context destruction.
/// resetting masker in the logger at the moment of
/// context destruction can't be done w/o synchronization / locks in a safe manner.
///
/// logger is Poco derived and i didn't want to brake it's interface,
/// also logger can be dynamically reconfigured without server restart,
/// and it actually recreates OwnSplitChannel when reconfiguration happen,
/// so that makes it's quite tricky. So it a bad candidate for owning masker too.
namespace DB
{
class SensitiveDataMasker
@ -20,6 +45,7 @@ class SensitiveDataMasker
private:
class MaskingRule;
std::vector<std::unique_ptr<MaskingRule>> all_masking_rules;
static std::unique_ptr<SensitiveDataMasker> sensitive_data_masker;
public:
SensitiveDataMasker(const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix);
@ -28,6 +54,11 @@ public:
/// Returns the number of matched rules.
size_t wipeSensitiveData(std::string & data) const;
/// setInstance is not thread-safe and should be called once in single-thread mode.
/// https://github.com/yandex/ClickHouse/pull/6810#discussion_r321183367
static void setInstance(std::unique_ptr<SensitiveDataMasker> sensitive_data_masker_);
static SensitiveDataMasker * getInstance();
/// Used in tests.
void addMaskingRule(const std::string & name, const std::string & regexp_string, const std::string & replacement_string);

View File

@ -0,0 +1,89 @@
#pragma once
#include <algorithm>
#include <Core/Block.h>
#include <Columns/IColumn.h>
#include <boost/smart_ptr/intrusive_ptr.hpp>
namespace DB
{
/// Allows you refer to the row in the block and hold the block ownership,
/// and thus avoid creating a temporary row object.
/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`;
/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece;
/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr;
/// The reference counter is not atomic, since it is used from one thread.
namespace detail
{
struct SharedBlock : Block
{
int refcount = 0;
ColumnRawPtrs all_columns;
ColumnRawPtrs sort_columns;
SharedBlock(Block && block) : Block(std::move(block)) {}
};
}
inline void intrusive_ptr_add_ref(detail::SharedBlock * ptr)
{
++ptr->refcount;
}
inline void intrusive_ptr_release(detail::SharedBlock * ptr)
{
if (0 == --ptr->refcount)
delete ptr;
}
using SharedBlockPtr = boost::intrusive_ptr<detail::SharedBlock>;
struct SharedBlockRowRef
{
ColumnRawPtrs * columns = nullptr;
size_t row_num;
SharedBlockPtr shared_block;
void swap(SharedBlockRowRef & other)
{
std::swap(columns, other.columns);
std::swap(row_num, other.row_num);
std::swap(shared_block, other.shared_block);
}
/// The number and types of columns must match.
bool operator==(const SharedBlockRowRef & other) const
{
size_t size = columns->size();
for (size_t i = 0; i < size; ++i)
if (0 != (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1))
return false;
return true;
}
bool operator!=(const SharedBlockRowRef & other) const
{
return !(*this == other);
}
void reset()
{
SharedBlockRowRef empty;
swap(empty);
}
bool empty() const { return columns == nullptr; }
size_t size() const { return empty() ? 0 : columns->size(); }
void set(SharedBlockPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_)
{
shared_block = shared_block_;
columns = columns_;
row_num = row_num_;
}
};
}

View File

@ -49,15 +49,26 @@ TraceCollector::TraceCollector(std::shared_ptr<TraceLog> & trace_log_)
#if !defined(__FreeBSD__)
/** Increase pipe size to avoid slowdown during fine-grained trace collection.
*/
constexpr int max_pipe_capacity_to_set = 1048576;
int pipe_size = fcntl(trace_pipe.fds_rw[1], F_GETPIPE_SZ);
if (-1 == pipe_size)
throwFromErrno("Cannot get pipe capacity", ErrorCodes::CANNOT_FCNTL);
for (errno = 0; errno != EPERM && pipe_size < max_pipe_capacity_to_set; pipe_size *= 2)
if (-1 == fcntl(trace_pipe.fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM)
throwFromErrno("Cannot increase pipe capacity to " + toString(pipe_size * 2), ErrorCodes::CANNOT_FCNTL);
{
if (errno == EINVAL)
{
LOG_INFO(log, "Cannot get pipe capacity, " << errnoToString(ErrorCodes::CANNOT_FCNTL) << ". Very old Linux kernels have no support for this fcntl.");
/// It will work nevertheless.
}
else
throwFromErrno("Cannot get pipe capacity", ErrorCodes::CANNOT_FCNTL);
}
else
{
constexpr int max_pipe_capacity_to_set = 1048576;
for (errno = 0; errno != EPERM && pipe_size < max_pipe_capacity_to_set; pipe_size *= 2)
if (-1 == fcntl(trace_pipe.fds_rw[1], F_SETPIPE_SZ, pipe_size * 2) && errno != EPERM)
throwFromErrno("Cannot increase pipe capacity to " + toString(pipe_size * 2), ErrorCodes::CANNOT_FCNTL);
LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, max_pipe_capacity_to_set)));
LOG_TRACE(log, "Pipe capacity is " << formatReadableSizeWithBinarySuffix(std::min(pipe_size, max_pipe_capacity_to_set)));
}
#endif
thread = ThreadFromGlobalPool(&TraceCollector::run, this);

View File

@ -0,0 +1,76 @@
#include <Common/parseGlobs.h>
#include <re2/re2.h>
#include <re2/stringpiece.h>
#include <algorithm>
#include <sstream>
namespace DB
{
/* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library fo matching
* with such steps:
* 1) search intervals like {0..9} and enums like {abc,xyz,qwe} in {}, replace them by regexp with pipe (expr1|expr2|expr3),
* 2) search and replace "*" and "?".
* Before each search need to escape symbols that we would not search.
*
* There are few examples in unit tests.
*/
std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs)
{
std::ostringstream oss_for_escaping;
/// Escaping only characters that not used in glob syntax
for (const auto & letter : initial_str_with_globs)
{
if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')'))
oss_for_escaping << '\\';
oss_for_escaping << letter;
}
std::string escaped_with_globs = oss_for_escaping.str();
static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*,
re2::StringPiece input(escaped_with_globs);
re2::StringPiece matched;
std::ostringstream oss_for_replacing;
size_t current_index = 0;
while (RE2::FindAndConsume(&input, enum_or_range, &matched))
{
std::string buffer = matched.ToString();
oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '(';
if (buffer.find(',') == std::string::npos)
{
size_t range_begin, range_end;
char point;
std::istringstream iss_range(buffer);
iss_range >> range_begin >> point >> point >> range_end;
oss_for_replacing << range_begin;
for (size_t i = range_begin + 1; i <= range_end; ++i)
{
oss_for_replacing << '|' << i;
}
}
else
{
std::replace(buffer.begin(), buffer.end(), ',', '|');
oss_for_replacing << buffer;
}
oss_for_replacing << ")";
current_index = input.data() - escaped_with_globs.data();
}
oss_for_replacing << escaped_with_globs.substr(current_index);
std::string almost_res = oss_for_replacing.str();
std::ostringstream oss_final_processing;
for (const auto & letter : almost_res)
{
if ((letter == '?') || (letter == '*'))
{
oss_final_processing << "[^/]"; /// '?' is any symbol except '/'
if (letter == '?')
continue;
}
if ((letter == '.') || (letter == '{') || (letter == '}'))
oss_final_processing << '\\';
oss_final_processing << letter;
}
return oss_final_processing.str();
}
}

View File

@ -0,0 +1,10 @@
#pragma once
#include <string>
#include <vector>
namespace DB
{
/* Parse globs in string and make a regexp for it.
*/
std::string makeRegexpPatternFromGlobs(const std::string & path);
}

View File

@ -0,0 +1,20 @@
#include <Common/parseGlobs.h>
#include <re2/re2.h>
#include <gtest/gtest.h>
using namespace DB;
TEST(Common, makeRegexpPatternFromGlobs)
{
EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..09}"), "f(1|2|3|4|5|6|7|8|9)");
EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..9}"), "f(1|2|3|4|5|6|7|8|9)");
EXPECT_EQ(makeRegexpPatternFromGlobs("f{0001..0000009}"), "f(1|2|3|4|5|6|7|8|9)");
EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..2}{1..2}"), "f(1|2)(1|2)");
EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..1}{1..1}"), "f(1)(1)");
EXPECT_EQ(makeRegexpPatternFromGlobs("f{0..0}{0..0}"), "f(0)(0)");
EXPECT_EQ(makeRegexpPatternFromGlobs("file{1..5}"),"file(1|2|3|4|5)");
EXPECT_EQ(makeRegexpPatternFromGlobs("file{1,2,3}"),"file(1|2|3)");
EXPECT_EQ(makeRegexpPatternFromGlobs("{1,2,3}blabla{a.x,b.x,c.x}smth[]_else{aa,bb}?*"), "(1|2|3)blabla(a\\.x|b\\.x|c\\.x)smth\\[\\]_else(aa|bb)[^/][^/]*");
}

View File

@ -59,7 +59,6 @@ TEST(Common, SensitiveDataMasker)
"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();
@ -86,7 +85,7 @@ TEST(Common, SensitiveDataMasker)
maskerbad.addMaskingRule("bad regexp", "**", "");
ADD_FAILURE() << "addMaskingRule() should throw an error" << std::endl;
}
catch (DB::Exception & e)
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
@ -170,7 +169,7 @@ TEST(Common, SensitiveDataMasker)
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (DB::Exception & e)
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
@ -178,7 +177,6 @@ TEST(Common, SensitiveDataMasker)
EXPECT_EQ(e.code(), DB::ErrorCodes::INVALID_CONFIG_PARAMETER);
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
@ -193,7 +191,7 @@ TEST(Common, SensitiveDataMasker)
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (DB::Exception & e)
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.what()),
@ -201,7 +199,6 @@ TEST(Common, SensitiveDataMasker)
EXPECT_EQ(e.code(), DB::ErrorCodes::NO_ELEMENTS_IN_CONFIG);
}
try
{
std::istringstream xml_isteam_bad(R"END(<?xml version="1.0"?>
@ -216,7 +213,7 @@ TEST(Common, SensitiveDataMasker)
ADD_FAILURE() << "XML should throw an error on bad XML" << std::endl;
}
catch (DB::Exception & e)
catch (const DB::Exception & e)
{
EXPECT_EQ(
std::string(e.message()),

View File

@ -39,7 +39,9 @@
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES 3
/// each period reduces the error counter by 2 times
/// too short a period can cause errors to disappear immediately after creation.
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2 * DBMS_DEFAULT_SEND_TIMEOUT_SEC)
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD 60
/// replica error max cap, this is to prevent replica from accumulating too many errors and taking to long to recover.
#define DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT 1000
#define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032
#define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058

View File

@ -16,4 +16,7 @@ using NameOrderedSet = std::set<std::string>;
using NameToNameMap = std::unordered_map<std::string, std::string>;
using NameToNameSetMap = std::unordered_map<std::string, NameSet>;
using NameWithAlias = std::pair<std::string, std::string>;
using NamesWithAliases = std::vector<NameWithAlias>;
}

View File

@ -348,6 +348,9 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, check_query_single_value_result, true, "Return check query result as single 1/0 value") \
M(SettingBool, allow_drop_detached, false, "Allow ALTER TABLE ... DROP DETACHED PART[ITION] ... queries") \
\
M(SettingSeconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.") \
M(SettingUInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up increadible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.") \
\
M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.") \
M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.") \
M(SettingSeconds, temporary_live_view_timeout, DEFAULT_TEMPORARY_LIVE_VIEW_TIMEOUT_SEC, "Timeout after which temporary live view is deleted.") \

View File

@ -4,13 +4,22 @@
#include <memory>
#include <cstddef>
#include <string>
#include <Core/Field.h>
class Collator;
namespace DB
{
struct FillColumnDescription
{
/// All missed values in range [FROM, TO) will be filled
/// Range [FROM, TO) respects sorting direction
Field fill_from; /// Fill value >= FILL_FROM
Field fill_to; /// Fill value + STEP < FILL_TO
Field fill_step; /// Default = 1 or -1 according to direction
};
/// Description of the sorting rule by one column.
struct SortColumnDescription
{
@ -20,12 +29,23 @@ struct SortColumnDescription
int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less.
/// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite.
std::shared_ptr<Collator> collator; /// Collator for locale-specific comparison of strings
bool with_fill;
FillColumnDescription fill_description;
SortColumnDescription(size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
: column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {}
SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr<Collator> & collator_ = nullptr)
: column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {}
SortColumnDescription(
size_t column_number_, int direction_, int nulls_direction_,
const std::shared_ptr<Collator> & collator_ = nullptr, bool with_fill_ = false,
const FillColumnDescription & fill_description_ = {})
: column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_)
, with_fill(with_fill_), fill_description(fill_description_) {}
SortColumnDescription(
const std::string & column_name_, int direction_, int nulls_direction_,
const std::shared_ptr<Collator> & collator_ = nullptr, bool with_fill_ = false,
const FillColumnDescription & fill_description_ = {})
: column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_)
, collator(collator_), with_fill(with_fill_), fill_description(fill_description_) {}
bool operator == (const SortColumnDescription & other) const
{

View File

@ -50,8 +50,8 @@ private:
std::vector<ColumnAggregateFunction *> columns_to_aggregate;
std::vector<SimpleAggregateDescription> columns_to_simple_aggregate;
RowRef current_key; /// The current primary key.
RowRef next_key; /// The primary key of the next row.
SharedBlockRowRef current_key; /// The current primary key.
SharedBlockRowRef next_key; /// The primary key of the next row.
/** We support two different cursors - with Collation and without.
* Templates are used instead of polymorphic SortCursor and calls to virtual functions.

View File

@ -47,12 +47,12 @@ private:
/// Read is finished.
bool finished = false;
RowRef current_key; /// The current primary key.
RowRef next_key; /// The primary key of the next row.
SharedBlockRowRef current_key; /// The current primary key.
SharedBlockRowRef next_key; /// The primary key of the next row.
RowRef first_negative; /// The first negative row for the current primary key.
RowRef last_positive; /// The last positive row for the current primary key.
RowRef last_negative; /// Last negative row. It is only stored if there is not one row is written to output.
SharedBlockRowRef first_negative; /// The first negative row for the current primary key.
SharedBlockRowRef last_positive; /// The last positive row for the current primary key.
SharedBlockRowRef last_negative; /// Last negative row. It is only stored if there is not one row is written to output.
size_t count_positive = 0; /// The number of positive rows for the current primary key.
size_t count_negative = 0; /// The number of negative rows for the current primary key.

View File

@ -0,0 +1,186 @@
#include <DataStreams/FillingBlockInputStream.h>
#include <Interpreters/convertFieldToType.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_WITH_FILL_EXPRESSION;
}
FillingBlockInputStream::FillingBlockInputStream(
const BlockInputStreamPtr & input, const SortDescription & sort_description_)
: sort_description(sort_description_), filling_row(sort_description_), next_row(sort_description_)
{
children.push_back(input);
header = children.at(0)->getHeader();
std::vector<bool> is_fill_column(header.columns());
for (const auto & elem : sort_description)
is_fill_column[header.getPositionByName(elem.column_name)] = true;
auto try_convert_fields = [](FillColumnDescription & descr, const DataTypePtr & type)
{
auto max_type = Field::Types::Null;
WhichDataType which(type);
DataTypePtr to_type;
if (isInteger(type) || which.isDateOrDateTime())
{
max_type = Field::Types::Int64;
to_type = std::make_shared<DataTypeInt64>();
}
else if (which.isFloat())
{
max_type = Field::Types::Float64;
to_type = std::make_shared<DataTypeFloat64>();
}
if (descr.fill_from.getType() > max_type || descr.fill_to.getType() > max_type
|| descr.fill_step.getType() > max_type)
return false;
descr.fill_from = convertFieldToType(descr.fill_from, *to_type);
descr.fill_to = convertFieldToType(descr.fill_to, *to_type);
descr.fill_step = convertFieldToType(descr.fill_step, *to_type);
return true;
};
for (size_t i = 0; i < header.columns(); ++i)
{
if (is_fill_column[i])
{
size_t pos = fill_column_positions.size();
auto & descr = filling_row.getFillDescription(pos);
auto type = header.getByPosition(i).type;
if (!try_convert_fields(descr, type))
throw Exception("Incompatible types of WITH FILL expression values with column type "
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (type->isValueRepresentedByUnsignedInteger() &&
((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) ||
(!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1))))
{
throw Exception("WITH FILL bound values cannot be negative for unsigned type "
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
fill_column_positions.push_back(i);
}
else
other_column_positions.push_back(i);
}
}
Block FillingBlockInputStream::readImpl()
{
Columns old_fill_columns;
Columns old_other_columns;
MutableColumns res_fill_columns;
MutableColumns res_other_columns;
auto init_columns_by_positions = [](const Block & block, Columns & columns,
MutableColumns & mutable_columns, const Positions & positions)
{
for (size_t pos : positions)
{
auto column = block.getByPosition(pos).column;
columns.push_back(column);
mutable_columns.push_back(column->cloneEmpty()->assumeMutable());
}
};
auto block = children.back()->read();
if (!block)
{
init_columns_by_positions(header, old_fill_columns, res_fill_columns, fill_column_positions);
init_columns_by_positions(header, old_other_columns, res_other_columns, other_column_positions);
bool should_insert_first = next_row < filling_row;
bool generated = false;
for (size_t i = 0; i < filling_row.size(); ++i)
next_row[i] = filling_row.getFillDescription(i).fill_to;
if (should_insert_first && filling_row < next_row)
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
while (filling_row.next(next_row))
{
generated = true;
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
}
if (generated)
return createResultBlock(res_fill_columns, res_other_columns);
return block;
}
size_t rows = block.rows();
init_columns_by_positions(block, old_fill_columns, res_fill_columns, fill_column_positions);
init_columns_by_positions(block, old_other_columns, res_other_columns, other_column_positions);
if (first)
{
for (size_t i = 0; i < filling_row.size(); ++i)
{
auto current_value = (*old_fill_columns[i])[0];
const auto & fill_from = filling_row.getFillDescription(i).fill_from;
if (!fill_from.isNull() && !equals(current_value, fill_from))
{
filling_row.initFromDefaults(i);
if (less(fill_from, current_value, filling_row.getDirection(i)))
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
break;
}
filling_row[i] = current_value;
}
first = false;
}
for (size_t row_ind = 0; row_ind < rows; ++row_ind)
{
bool should_insert_first = next_row < filling_row;
for (size_t i = 0; i < filling_row.size(); ++i)
{
auto current_value = (*old_fill_columns[i])[row_ind];
const auto & fill_to = filling_row.getFillDescription(i).fill_to;
if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i)))
next_row[i] = current_value;
else
next_row[i] = fill_to;
}
/// A case, when at previous step row was initialized from defaults 'fill_from' values
/// and probably we need to insert it to block.
if (should_insert_first && filling_row < next_row)
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
/// Insert generated filling row to block, while it is less than current row in block.
while (filling_row.next(next_row))
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind);
copyRowFromColumns(res_other_columns, old_other_columns, row_ind);
}
return createResultBlock(res_fill_columns, res_other_columns);
}
Block FillingBlockInputStream::createResultBlock(MutableColumns & fill_columns, MutableColumns & other_columns) const
{
MutableColumns result_columns(header.columns());
for (size_t i = 0; i < fill_columns.size(); ++i)
result_columns[fill_column_positions[i]] = std::move(fill_columns[i]);
for (size_t i = 0; i < other_columns.size(); ++i)
result_columns[other_column_positions[i]] = std::move(other_columns[i]);
return header.cloneWithColumns(std::move(result_columns));
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <DataStreams/IBlockInputStream.h>
#include <Interpreters/FillingRow.h>
namespace DB
{
/** Implements modifier WITH FILL of ORDER BY clause.
* It fills gaps in data stream by rows with missing values in columns with set WITH FILL and deafults in other columns.
* Optionally FROM, TO and STEP values can be specified.
*/
class FillingBlockInputStream : public IBlockInputStream
{
public:
FillingBlockInputStream(const BlockInputStreamPtr & input, const SortDescription & fill_description_);
String getName() const override { return "Filling"; }
Block getHeader() const override { return header; }
protected:
Block readImpl() override;
private:
Block createResultBlock(MutableColumns & fill_columns, MutableColumns & other_columns) const;
const SortDescription sort_description; /// Contains only rows with WITH FILL.
FillingRow filling_row; /// Current row, which is used to fill gaps.
FillingRow next_row; /// Row to which we need to generate filling rows.
Block header;
using Positions = std::vector<size_t>;
Positions fill_column_positions;
Positions other_column_positions;
bool first = true;
};
}

View File

@ -321,7 +321,7 @@ void GraphiteRollupSortedBlockInputStream::finishCurrentGroup(MutableColumns & m
}
void GraphiteRollupSortedBlockInputStream::accumulateRow(RowRef & row)
void GraphiteRollupSortedBlockInputStream::accumulateRow(SharedBlockRowRef & row)
{
const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule);
if (aggregate_state_created)

View File

@ -204,7 +204,7 @@ private:
StringRef current_group_path;
/// Last row with maximum version for current primary key (time bucket).
RowRef current_subgroup_newest_row;
SharedBlockRowRef current_subgroup_newest_row;
/// Time of last read row
time_t current_time = 0;
@ -236,7 +236,7 @@ private:
void finishCurrentGroup(MutableColumns & merged_columns);
/// Update the state of the aggregate function with the new `value`.
void accumulateRow(RowRef & row);
void accumulateRow(SharedBlockRowRef & row);
};
}

View File

@ -16,11 +16,12 @@ namespace DB
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int INVALID_USAGE_OF_INPUT;
}
InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, const Context & context)
const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, const Context & context, const ASTPtr & input_function)
{
const auto * ast_insert_query = ast->as<ASTInsertQuery>();
@ -29,7 +30,11 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
String format = ast_insert_query->format;
if (format.empty())
{
if (input_function)
throw Exception("FORMAT must be specified for function input()", ErrorCodes::INVALID_USAGE_OF_INPUT);
format = "Values";
}
/// Data could be in parsed (ast_insert_query.data) and in not parsed yet (input_buffer_tail_part) part of query.
@ -51,7 +56,7 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
res_stream = context.getInputFormat(format, *input_buffer_contacenated, header, context.getSettings().max_insert_block_size);
if (context.getSettingsRef().input_format_defaults_for_omitted_fields && !ast_insert_query->table.empty())
if (context.getSettingsRef().input_format_defaults_for_omitted_fields && !ast_insert_query->table.empty() && !input_function)
{
StoragePtr storage = context.getTable(ast_insert_query->database, ast_insert_query->table);
auto column_defaults = storage->getColumns().getDefaults();

View File

@ -19,7 +19,11 @@ class Context;
class InputStreamFromASTInsertQuery : public IBlockInputStream
{
public:
InputStreamFromASTInsertQuery(const ASTPtr & ast, ReadBuffer * input_buffer_tail_part, const Block & header, const Context & context);
InputStreamFromASTInsertQuery(const ASTPtr & ast,
ReadBuffer * input_buffer_tail_part,
const Block & header,
const Context & context,
const ASTPtr & input_function);
Block readImpl() override { return res_stream->read(); }
void readPrefixImpl() override { return res_stream->readPrefix(); }

View File

@ -6,8 +6,30 @@
namespace DB
{
LimitBlockInputStream::LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_, bool use_limit_as_total_rows_approx)
: limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_)
/// gets pointers to all columns of block, which were used for ORDER BY
static ColumnRawPtrs extractSortColumns(const Block & block, const SortDescription & description)
{
size_t size = description.size();
ColumnRawPtrs res;
res.reserve(size);
for (size_t i = 0; i < size; ++i)
{
const IColumn * column = !description[i].column_name.empty()
? block.getByName(description[i].column_name).column.get()
: block.safeGetByPosition(description[i].column_number).column.get();
res.emplace_back(column);
}
return res;
}
LimitBlockInputStream::LimitBlockInputStream(
const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_,
bool use_limit_as_total_rows_approx, bool with_ties_, const SortDescription & description_)
: limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_), with_ties(with_ties_)
, description(description_)
{
if (use_limit_as_total_rows_approx)
{
@ -17,13 +39,45 @@ LimitBlockInputStream::LimitBlockInputStream(const BlockInputStreamPtr & input,
children.push_back(input);
}
Block LimitBlockInputStream::readImpl()
{
Block res;
UInt64 rows = 0;
/// pos - how many rows were read, including the last read block
/// pos >= offset + limit and all rows in the end of previous block were equal
/// to row at 'limit' position. So we check current block.
if (!ties_row_ref.empty() && pos >= offset + limit)
{
res = children.back()->read();
rows = res.rows();
if (!res)
return res;
SharedBlockPtr ptr = new detail::SharedBlock(std::move(res));
ptr->sort_columns = extractSortColumns(*ptr, description);
UInt64 len;
for (len = 0; len < rows; ++len)
{
SharedBlockRowRef current_row;
current_row.set(ptr, &ptr->sort_columns, len);
if (current_row != ties_row_ref)
{
ties_row_ref.reset();
break;
}
}
if (len < rows)
{
for (size_t i = 0; i < ptr->columns(); ++i)
ptr->safeGetByPosition(i).column = ptr->safeGetByPosition(i).column->cut(0, len);
}
return *ptr;
}
if (pos >= offset + limit)
{
@ -46,9 +100,18 @@ Block LimitBlockInputStream::readImpl()
pos += rows;
} while (pos <= offset);
/// return the whole block
SharedBlockPtr ptr = new detail::SharedBlock(std::move(res));
if (with_ties)
ptr->sort_columns = extractSortColumns(*ptr, description);
/// give away the whole block
if (pos >= offset + rows && pos <= offset + limit)
return res;
{
/// Save rowref for last row, because probalbly next block begins with the same row.
if (with_ties && pos == offset + limit)
ties_row_ref.set(ptr, &ptr->sort_columns, rows - 1);
return *ptr;
}
/// give away a piece of the block
UInt64 start = std::max(
@ -60,13 +123,36 @@ Block LimitBlockInputStream::readImpl()
static_cast<Int64>(pos) - static_cast<Int64>(offset),
static_cast<Int64>(limit) + static_cast<Int64>(offset) - static_cast<Int64>(pos) + static_cast<Int64>(rows)));
for (size_t i = 0; i < res.columns(); ++i)
res.getByPosition(i).column = res.getByPosition(i).column->cut(start, length);
/// check if other rows in current block equals to last one in limit
if (with_ties)
{
ties_row_ref.set(ptr, &ptr->sort_columns, start + length - 1);
for (size_t i = ties_row_ref.row_num + 1; i < rows; ++i)
{
SharedBlockRowRef current_row;
current_row.set(ptr, &ptr->sort_columns, i);
if (current_row == ties_row_ref)
++length;
else
{
ties_row_ref.reset();
break;
}
}
}
if (length == rows)
return *ptr;
for (size_t i = 0; i < ptr->columns(); ++i)
ptr->safeGetByPosition(i).column = ptr->safeGetByPosition(i).column->cut(start, length);
// TODO: we should provide feedback to child-block, so it will know how many rows are actually consumed.
// It's crucial for streaming engines like Kafka.
return res;
return *ptr;
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <DataStreams/IBlockInputStream.h>
#include <Common/SharedBlockRowRef.h>
namespace DB
@ -17,8 +18,13 @@ public:
* If always_read_till_end = true - reads all the data to the end, but ignores them. This is necessary in rare cases:
* when otherwise, due to the cancellation of the request, we would not have received the data for GROUP BY WITH TOTALS from the remote server.
* If use_limit_as_total_rows_approx = true, then addTotalRowsApprox is called to use the limit in progress & stats
* with_ties = true, when query has WITH TIES modifier. If so, description should be provided
* description lets us know which row we should check for equality
*/
LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_ = false, bool use_limit_as_total_rows_approx = false);
LimitBlockInputStream(
const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_,
bool always_read_till_end_ = false, bool use_limit_as_total_rows_approx = false,
bool with_ties_ = false, const SortDescription & description_ = {});
String getName() const override { return "Limit"; }
@ -32,6 +38,9 @@ private:
UInt64 offset;
UInt64 pos = 0;
bool always_read_till_end;
bool with_ties;
const SortDescription description;
SharedBlockRowRef ties_row_ref;
};
}

View File

@ -5,6 +5,7 @@
#include <boost/smart_ptr/intrusive_ptr.hpp>
#include <common/logger_useful.h>
#include <Common/SharedBlockRowRef.h>
#include <Core/Row.h>
#include <Core/SortDescription.h>
@ -24,39 +25,6 @@ namespace ErrorCodes
}
/// Allows you refer to the row in the block and hold the block ownership,
/// and thus avoid creating a temporary row object.
/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`;
/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece;
/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr;
/// The reference counter is not atomic, since it is used from one thread.
namespace detail
{
struct SharedBlock : Block
{
int refcount = 0;
ColumnRawPtrs all_columns;
ColumnRawPtrs sort_columns;
SharedBlock(Block && block) : Block(std::move(block)) {}
};
}
using SharedBlockPtr = boost::intrusive_ptr<detail::SharedBlock>;
inline void intrusive_ptr_add_ref(detail::SharedBlock * ptr)
{
++ptr->refcount;
}
inline void intrusive_ptr_release(detail::SharedBlock * ptr)
{
if (0 == --ptr->refcount)
delete ptr;
}
/** Merges several sorted streams into one sorted stream.
*/
class MergingSortedBlockInputStream : public IBlockInputStream
@ -78,44 +46,6 @@ public:
Block getHeader() const override { return header; }
protected:
struct RowRef
{
ColumnRawPtrs * columns = nullptr;
size_t row_num = 0;
SharedBlockPtr shared_block;
void swap(RowRef & other)
{
std::swap(columns, other.columns);
std::swap(row_num, other.row_num);
std::swap(shared_block, other.shared_block);
}
/// The number and types of columns must match.
bool operator==(const RowRef & other) const
{
size_t size = columns->size();
for (size_t i = 0; i < size; ++i)
if (0 != (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1))
return false;
return true;
}
bool operator!=(const RowRef & other) const
{
return !(*this == other);
}
void reset()
{
RowRef empty;
swap(empty);
}
bool empty() const { return columns == nullptr; }
size_t size() const { return empty() ? 0 : columns->size(); }
};
/// Simple class, which allows to check stop condition during merge process
/// in simple case it just compare amount of merged rows with max_block_size
/// in `count_average` case it compares amount of merged rows with linear combination
@ -148,7 +78,6 @@ protected:
}
};
Block readImpl() override;
void readSuffixImpl() override;
@ -230,7 +159,7 @@ protected:
}
template <typename TSortCursor>
void setRowRef(RowRef & row_ref, TSortCursor & cursor)
void setRowRef(SharedBlockRowRef & row_ref, TSortCursor & cursor)
{
row_ref.row_num = cursor.impl->pos;
row_ref.shared_block = source_blocks[cursor.impl->order];
@ -238,7 +167,7 @@ protected:
}
template <typename TSortCursor>
void setPrimaryKeyRef(RowRef & row_ref, TSortCursor & cursor)
void setPrimaryKeyRef(SharedBlockRowRef & row_ref, TSortCursor & cursor)
{
row_ref.row_num = cursor.impl->pos;
row_ref.shared_block = source_blocks[cursor.impl->order];

View File

@ -41,11 +41,11 @@ private:
bool finished = false;
/// Primary key of current row.
RowRef current_key;
SharedBlockRowRef current_key;
/// Primary key of next row.
RowRef next_key;
SharedBlockRowRef next_key;
/// Last row with maximum version for current primary key.
RowRef selected_row;
SharedBlockRowRef selected_row;
/// The position (into current_row_sources) of the row with the highest version.
size_t max_pos = 0;

View File

@ -129,8 +129,8 @@ private:
std::vector<AggregateDescription> columns_to_aggregate;
std::vector<MapDescription> maps_to_sum;
RowRef current_key; /// The current primary key.
RowRef next_key; /// The primary key of the next row.
SharedBlockRowRef current_key; /// The current primary key.
SharedBlockRowRef next_key; /// The primary key of the next row.
Row current_row;
bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally.

View File

@ -47,7 +47,7 @@ void VersionedCollapsingSortedBlockInputStream::insertGap(size_t gap_size)
}
}
void VersionedCollapsingSortedBlockInputStream::insertRow(size_t skip_rows, const RowRef & row, MutableColumns & merged_columns)
void VersionedCollapsingSortedBlockInputStream::insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns)
{
const auto & columns = row.shared_block->all_columns;
for (size_t i = 0; i < num_columns; ++i)
@ -111,7 +111,7 @@ void VersionedCollapsingSortedBlockInputStream::merge(MutableColumns & merged_co
SortCursor current = queue.top();
size_t current_block_granularity = current->rows;
RowRef next_key;
SharedBlockRowRef next_key;
Int8 sign = assert_cast<const ColumnInt8 &>(*current->all_columns[sign_column_number]).getData()[current->pos];

View File

@ -197,7 +197,7 @@ private:
Int8 sign_in_queue = 0;
const size_t max_rows_in_queue;
/// Rows with the same primary key and sign.
FixedSizeDequeWithGaps<RowRef> current_keys;
FixedSizeDequeWithGaps<SharedBlockRowRef> current_keys;
size_t blocks_written = 0;
@ -207,7 +207,7 @@ private:
void merge(MutableColumns & merged_columns, std::priority_queue<SortCursor> & queue);
/// Output to result row for the current primary key.
void insertRow(size_t skip_rows, const RowRef & row, MutableColumns & merged_columns);
void insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns);
void insertGap(size_t gap_size);
};

View File

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

View File

@ -858,7 +858,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument
builder.initSink(input_rows_count);
if (index == 0u)
throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
throw Exception("Array indices are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX);
if (!(executeNumberConst<UInt8>(block, arguments, result, index, builder)
|| executeNumberConst<UInt16>(block, arguments, result, index, builder)

View File

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

View File

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

View File

@ -1,4 +1,5 @@
#include <IO/HDFSCommon.h>
#include <Poco/URI.h>
#if USE_HDFS
#include <Common/Exception.h>
@ -11,8 +12,9 @@ extern const int BAD_ARGUMENTS;
extern const int NETWORK_ERROR;
}
HDFSBuilderPtr createHDFSBuilder(const Poco::URI & uri)
HDFSBuilderPtr createHDFSBuilder(const std::string & uri_str)
{
const Poco::URI uri(uri_str);
auto & host = uri.getHost();
auto port = uri.getPort();
auto & path = uri.getPath();

View File

@ -1,7 +1,6 @@
#include <Common/config.h>
#include <memory>
#include <type_traits>
#include <Poco/URI.h>
#if USE_HDFS
#include <hdfs/hdfs.h>
@ -27,12 +26,32 @@ struct HDFSFsDeleter
}
struct HDFSFileInfo
{
hdfsFileInfo * file_info;
int length;
HDFSFileInfo()
: file_info(nullptr)
, length(0)
{
}
HDFSFileInfo(const HDFSFileInfo & other) = delete;
HDFSFileInfo(HDFSFileInfo && other) = default;
HDFSFileInfo & operator=(const HDFSFileInfo & other) = delete;
HDFSFileInfo & operator=(HDFSFileInfo && other) = default;
~HDFSFileInfo()
{
hdfsFreeFileInfo(file_info, length);
}
};
using HDFSBuilderPtr = std::unique_ptr<hdfsBuilder, detail::HDFSBuilderDeleter>;
using HDFSFSPtr = std::unique_ptr<std::remove_pointer_t<hdfsFS>, detail::HDFSFsDeleter>;
// set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large
/// TODO Allow to tune from query Settings.
HDFSBuilderPtr createHDFSBuilder(const Poco::URI & hdfs_uri);
HDFSBuilderPtr createHDFSBuilder(const std::string & hdfs_uri);
HDFSFSPtr createHDFSFS(hdfsBuilder * builder);
}
#endif

View File

@ -2,7 +2,6 @@
#if USE_HDFS
#include <IO/HDFSCommon.h>
#include <Poco/URI.h>
#include <hdfs/hdfs.h>
@ -16,7 +15,7 @@ namespace ErrorCodes
struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
{
Poco::URI hdfs_uri;
std::string hdfs_uri;
hdfsFile fin;
HDFSBuilderPtr builder;
HDFSFSPtr fs;
@ -26,8 +25,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
, builder(createHDFSBuilder(hdfs_uri))
, fs(createHDFSFS(builder.get()))
{
auto & path = hdfs_uri.getPath();
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
const std::string path = hdfs_uri.substr(begin_of_path);
fin = hdfsOpenFile(fs.get(), path.c_str(), O_RDONLY, 0, 0, 0);
if (fin == nullptr)
@ -39,7 +38,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl
{
int bytes_read = hdfsRead(fs.get(), fin, start, size);
if (bytes_read < 0)
throw Exception("Fail to read HDFS file: " + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()),
throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()),
ErrorCodes::NETWORK_ERROR);
return bytes_read;
}

View File

@ -2,7 +2,6 @@
#if USE_HDFS
#include <Poco/URI.h>
#include <IO/WriteBufferFromHDFS.h>
#include <IO/HDFSCommon.h>
#include <hdfs/hdfs.h>
@ -21,7 +20,7 @@ extern const int CANNOT_FSYNC;
struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
{
Poco::URI hdfs_uri;
std::string hdfs_uri;
hdfsFile fout;
HDFSBuilderPtr builder;
HDFSFSPtr fs;
@ -31,7 +30,11 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
, builder(createHDFSBuilder(hdfs_uri))
, fs(createHDFSFS(builder.get()))
{
auto & path = hdfs_uri.getPath();
const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2);
const std::string path = hdfs_uri.substr(begin_of_path);
if (path.find("*?{") != std::string::npos)
throw Exception("URI '" + hdfs_uri + "' contains globs, so the table is in readonly mode", ErrorCodes::CANNOT_OPEN_FILE);
fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0);
if (fout == nullptr)
@ -52,7 +55,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
{
int bytes_written = hdfsWrite(fs.get(), fout, start, size);
if (bytes_written < 0)
throw Exception("Fail to write HDFS file: " + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()),
throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()),
ErrorCodes::NETWORK_ERROR);
return bytes_written;
}
@ -61,7 +64,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl
{
int result = hdfsSync(fs.get(), fout);
if (result < 0)
throwFromErrno("Cannot HDFS sync" + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()),
throwFromErrno("Cannot HDFS sync" + hdfs_uri + " " + std::string(hdfsGetLastError()),
ErrorCodes::CANNOT_FSYNC);
}
};

View File

@ -93,14 +93,14 @@ NameSet AnalyzedJoin::getOriginalColumnsSet() const
return out;
}
std::unordered_map<String, String> AnalyzedJoin::getOriginalColumnsMap(const NameSet & required_columns) const
NamesWithAliases AnalyzedJoin::getNamesWithAliases(const NameSet & required_columns) const
{
std::unordered_map<String, String> out;
NamesWithAliases out;
for (const auto & column : required_columns)
{
auto it = original_names.find(column);
if (it != original_names.end())
out.insert(*it);
out.emplace_back(it->second, it->first); /// {original_name, name}
}
return out;
}
@ -129,15 +129,15 @@ Names AnalyzedJoin::requiredJoinedNames() const
return Names(required_columns_set.begin(), required_columns_set.end());
}
void AnalyzedJoin::appendRequiredColumns(const Block & sample, NameSet & required_columns) const
NamesWithAliases AnalyzedJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const
{
for (auto & column : key_names_right)
NameSet required_columns(action_required_columns.begin(), action_required_columns.end());
for (auto & column : requiredJoinedNames())
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);
return getNamesWithAliases(required_columns);
}
void AnalyzedJoin::addJoinedColumn(const NameAndTypePair & joined_column)

View File

@ -64,12 +64,12 @@ public:
NameSet getQualifiedColumnsSet() const;
NameSet getOriginalColumnsSet() const;
std::unordered_map<String, String> getOriginalColumnsMap(const NameSet & required_columns) const;
NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const;
NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_columns) const;
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;

View File

@ -347,7 +347,8 @@ Cluster::Cluster(const Poco::Util::AbstractConfiguration & config, const Setting
}
ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>(
all_replicas_pools, settings.load_balancing);
all_replicas_pools, settings.load_balancing,
settings.distributed_replica_error_half_life.totalSeconds(), settings.distributed_replica_error_cap);
if (weight)
slot_to_shard.insert(std::end(slot_to_shard), weight, shards_info.size());
@ -398,7 +399,8 @@ Cluster::Cluster(const Settings & settings, const std::vector<std::vector<String
}
ConnectionPoolWithFailoverPtr shard_pool = std::make_shared<ConnectionPoolWithFailover>(
all_replicas, settings.load_balancing);
all_replicas, settings.load_balancing,
settings.distributed_replica_error_half_life.totalSeconds(), settings.distributed_replica_error_cap);
slot_to_shard.insert(std::end(slot_to_shard), default_weight, shards_info.size());
shards_info.push_back({{}, current_shard_num, default_weight, std::move(shard_local_addresses), std::move(shard_pool),

View File

@ -30,7 +30,7 @@ Context removeUserRestrictionsFromSettings(const Context & context, const Settin
/// Set as unchanged to avoid sending to remote server.
new_settings.max_concurrent_queries_for_user.changed = false;
new_settings.max_memory_usage_for_user.changed = false;
new_settings.max_memory_usage_for_all_queries = false;
new_settings.max_memory_usage_for_all_queries.changed = false;
Context new_context(context);
new_context.setSettings(new_settings);

View File

@ -143,8 +143,6 @@ 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)
@ -287,8 +285,6 @@ struct ContextShared
/// Stop trace collector if any
trace_collector.reset();
sensitive_data_masker.reset();
}
bool hasTraceCollector()
@ -538,23 +534,6 @@ 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();
@ -1222,8 +1201,8 @@ void Context::setCurrentQueryId(const String & query_id)
} words;
} random;
random.words.a = thread_local_rng();
random.words.b = thread_local_rng();
random.words.a = thread_local_rng(); //-V656
random.words.b = thread_local_rng(); //-V656
/// Use protected constructor.
struct qUUID : Poco::UUID
@ -2053,6 +2032,51 @@ void Context::initializeExternalTablesIfSet()
}
void Context::setInputInitializer(InputInitializer && initializer)
{
if (input_initializer_callback)
throw Exception("Input initializer is already set", ErrorCodes::LOGICAL_ERROR);
input_initializer_callback = std::move(initializer);
}
void Context::initializeInput(const StoragePtr & input_storage)
{
if (!input_initializer_callback)
throw Exception("Input initializer is not set", ErrorCodes::LOGICAL_ERROR);
input_initializer_callback(*this, input_storage);
/// Reset callback
input_initializer_callback = {};
}
void Context::setInputBlocksReaderCallback(InputBlocksReader && reader)
{
if (input_blocks_reader)
throw Exception("Input blocks reader is already set", ErrorCodes::LOGICAL_ERROR);
input_blocks_reader = std::move(reader);
}
InputBlocksReader Context::getInputBlocksReaderCallback() const
{
return input_blocks_reader;
}
void Context::resetInputCallbacks()
{
if (input_initializer_callback)
input_initializer_callback = {};
if (input_blocks_reader)
input_blocks_reader = {};
}
SessionCleaner::~SessionCleaner()
{
try

View File

@ -12,7 +12,6 @@
#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>
@ -101,6 +100,11 @@ using TableAndCreateASTs = std::map<String, TableAndCreateAST>;
/// Callback for external tables initializer
using ExternalTablesInitializer = std::function<void(Context &)>;
/// Callback for initialize input()
using InputInitializer = std::function<void(Context &, const StoragePtr &)>;
/// Callback for reading blocks of data from client for function input()
using InputBlocksReader = std::function<Block(Context &)>;
/// An empty interface for an arbitrary object that may be attached by a shared pointer
/// to query context, when using ClickHouse as a library.
struct IHostContext
@ -125,6 +129,9 @@ private:
ClientInfo client_info;
ExternalTablesInitializer external_tables_initializer_callback;
InputInitializer input_initializer_callback;
InputBlocksReader input_blocks_reader;
std::shared_ptr<QuotaForIntervals> quota; /// Current quota. By default - empty quota, that have no limits.
String current_database;
Settings settings; /// Setting for query execution.
@ -178,9 +185,6 @@ 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);
@ -217,6 +221,17 @@ public:
/// This method is called in executeQuery() and will call the external tables initializer.
void initializeExternalTablesIfSet();
/// When input() is present we have to send columns structure to client
void setInputInitializer(InputInitializer && initializer);
/// This method is called in StorageInput::read while executing query
void initializeInput(const StoragePtr & input_storage);
/// Callback for read data blocks from client one by one for function input()
void setInputBlocksReaderCallback(InputBlocksReader && reader);
/// Get callback for reading data for input()
InputBlocksReader getInputBlocksReaderCallback() const;
void resetInputCallbacks();
ClientInfo & getClientInfo() { return client_info; }
const ClientInfo & getClientInfo() const { return client_info; }

View File

@ -1,6 +1,5 @@
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/AnalyzedJoin.h> /// for getNamesAndTypeListFromTableExpression
#include <Interpreters/Context.h>
#include <Common/typeid_cast.h>
@ -15,6 +14,8 @@ namespace DB
NameSet removeDuplicateColumns(NamesAndTypesList & columns);
struct ASTTableExpression;
NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context);
DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database)
{

View File

@ -20,9 +20,6 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
using NameWithAlias = std::pair<std::string, std::string>;
using NamesWithAliases = std::vector<NameWithAlias>;
class AnalyzedJoin;
class IPreparedFunction;

View File

@ -29,6 +29,7 @@
#include <Interpreters/PredicateExpressionsOptimizer.h>
#include <Interpreters/ExternalDictionaries.h>
#include <Interpreters/Set.h>
#include <Interpreters/AnalyzedJoin.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/parseAggregateFunctionParameters.h>
@ -417,8 +418,7 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b
if (!ast_join)
return false;
SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join);
syntax->analyzed_join->setHashJoin(subquery_for_set.join);
makeTableJoin(*ast_join);
initChain(chain, sourceColumns());
ExpressionActionsChain::Step & step = chain.steps.back();
@ -456,7 +456,15 @@ static JoinPtr tryGetStorageJoin(const ASTTablesInSelectQueryElement & join_elem
return {};
}
SubqueryForSet & SelectQueryExpressionAnalyzer::getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element)
static ExpressionActionsPtr createJoinedBlockActions(const Context & context, const AnalyzedJoin & analyzed_join)
{
ASTPtr expression_list = analyzed_join.rightKeysList();
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list,
analyzed_join.columnsFromJoinedTable(), analyzed_join.requiredJoinedNames());
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
}
void SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQueryElement & join_element)
{
/// Two JOINs are not supported with the same subquery, but different USINGs.
auto join_hash = join_element.getTreeHash();
@ -469,64 +477,45 @@ SubqueryForSet & SelectQueryExpressionAnalyzer::getSubqueryForJoin(const ASTTabl
subquery_for_set.join = tryGetStorageJoin(join_element, context);
if (!subquery_for_set.join)
makeHashJoin(join_element, subquery_for_set);
{
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(context, analyzedJoin());
return subquery_for_set;
if (!subquery_for_set.source)
makeSubqueryForJoin(join_element, joined_block_actions, subquery_for_set);
/// Test actions on sample block (early error detection)
Block sample_block = subquery_for_set.renamedSampleBlock();
joined_block_actions->execute(sample_block);
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join);
subquery_for_set.joined_block_actions = joined_block_actions;
}
syntax->analyzed_join->setHashJoin(subquery_for_set.join);
}
void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryElement & join_element,
SubqueryForSet & subquery_for_set) const
void SelectQueryExpressionAnalyzer::makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element,
const ExpressionActionsPtr & joined_block_actions,
SubqueryForSet & subquery_for_set) const
{
/// Actions which need to be calculated on joined block.
ExpressionActionsPtr joined_block_actions = createJoinedBlockActions();
/** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs
* - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1,
* in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`.
* - this function shows the expression JOIN _data1.
*/
if (!subquery_for_set.source)
{
ASTPtr table;
auto & table_to_join = join_element.table_expression->as<ASTTableExpression &>();
if (table_to_join.subquery)
table = table_to_join.subquery;
else if (table_to_join.table_function)
table = table_to_join.table_function;
else if (table_to_join.database_and_table_name)
table = table_to_join.database_and_table_name;
NamesWithAliases required_columns_with_aliases =
analyzedJoin().getRequiredColumns(joined_block_actions->getSampleBlock(), joined_block_actions->getRequiredColumns());
Names action_columns = joined_block_actions->getRequiredColumns();
NameSet required_columns(action_columns.begin(), action_columns.end());
Names original_columns;
for (auto & pr : required_columns_with_aliases)
original_columns.push_back(pr.first);
analyzedJoin().appendRequiredColumns(joined_block_actions->getSampleBlock(), required_columns);
auto interpreter = interpretSubquery(join_element.table_expression, context, subquery_depth, original_columns);
auto original_map = analyzedJoin().getOriginalColumnsMap(required_columns);
Names original_columns;
for (auto & pr : original_map)
original_columns.push_back(pr.second);
auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns);
subquery_for_set.makeSource(interpreter, original_map);
}
Block sample_block = subquery_for_set.renamedSampleBlock();
joined_block_actions->execute(sample_block);
/// TODO You do not need to set this up when JOIN is only needed on remote servers.
subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join);
subquery_for_set.joined_block_actions = joined_block_actions;
}
ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const
{
ASTPtr expression_list = analyzedJoin().rightKeysList();
Names required_columns = analyzedJoin().requiredJoinedNames();
auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columnsFromJoinedTable(), required_columns);
return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false);
subquery_for_set.makeSource(interpreter, std::move(required_columns_with_aliases));
}
bool SelectQueryExpressionAnalyzer::appendPrewhere(

View File

@ -26,9 +26,6 @@ class ASTExpressionList;
class ASTSelectQuery;
struct ASTTablesInSelectQueryElement;
struct SyntaxAnalyzerResult;
using SyntaxAnalyzerResultPtr = std::shared_ptr<const SyntaxAnalyzerResult>;
/// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately.
struct ExpressionAnalyzerData
{
@ -222,9 +219,9 @@ private:
*/
void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name);
SubqueryForSet & getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element);
ExpressionActionsPtr createJoinedBlockActions() const;
void makeHashJoin(const ASTTablesInSelectQueryElement & join_element, SubqueryForSet & subquery_for_set) const;
void makeTableJoin(const ASTTablesInSelectQueryElement & join_element);
void makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, const ExpressionActionsPtr & joined_block_actions,
SubqueryForSet & subquery_for_set) const;
const ASTSelectQuery * getAggregatingQuery() const;
};

View File

@ -0,0 +1,127 @@
#include <Interpreters/FillingRow.h>
namespace DB
{
bool less(const Field & lhs, const Field & rhs, int direction)
{
if (direction == -1)
return applyVisitor(FieldVisitorAccurateLess(), rhs, lhs);
return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs);
}
bool equals(const Field & lhs, const Field & rhs)
{
return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs);
}
FillingRow::FillingRow(const SortDescription & description_) : description(description_)
{
row.resize(description.size());
}
bool FillingRow::operator<(const FillingRow & other) const
{
for (size_t i = 0; i < size(); ++i)
{
if (row[i].isNull() || other[i].isNull() || equals(row[i], other[i]))
continue;
return less(row[i], other[i], getDirection(i));
}
return false;
}
bool FillingRow::operator==(const FillingRow & other) const
{
for (size_t i = 0; i < size(); ++i)
if (!equals(row[i], other[i]))
return false;
return true;
}
bool FillingRow::next(const FillingRow & to_row)
{
size_t pos = 0;
/// Find position we need to increment for generating next row.
for (; pos < row.size(); ++pos)
if (!row[pos].isNull() && !to_row[pos].isNull() && !equals(row[pos], to_row[pos]))
break;
if (pos == row.size() || less(to_row[pos], row[pos], getDirection(pos)))
return false;
/// If we have any 'fill_to' value at position greater than 'pos',
/// we need to generate rows up to 'fill_to' value.
for (size_t i = row.size() - 1; i > pos; --i)
{
if (getFillDescription(i).fill_to.isNull() || row[i].isNull())
continue;
auto next_value = row[i];
applyVisitor(FieldVisitorSum(getFillDescription(i).fill_step), next_value);
if (less(next_value, getFillDescription(i).fill_to, getDirection(i)))
{
row[i] = next_value;
initFromDefaults(i + 1);
return true;
}
}
auto next_value = row[pos];
applyVisitor(FieldVisitorSum(getFillDescription(pos).fill_step), next_value);
if (less(to_row[pos], next_value, getDirection(pos)))
return false;
row[pos] = next_value;
if (equals(row[pos], to_row[pos]))
{
bool is_less = false;
for (size_t i = pos + 1; i < size(); ++i)
{
const auto & fill_from = getFillDescription(i).fill_from;
if (!fill_from.isNull())
row[i] = fill_from;
else
row[i] = to_row[i];
is_less |= less(row[i], to_row[i], getDirection(i));
}
return is_less;
}
initFromDefaults(pos + 1);
return true;
}
void FillingRow::initFromDefaults(size_t from_pos)
{
for (size_t i = from_pos; i < row.size(); ++i)
row[i] = getFillDescription(i).fill_from;
}
void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row)
{
for (size_t i = 0; i < filling_columns.size(); ++i)
{
if (filling_row[i].isNull())
filling_columns[i]->insertDefault();
else
filling_columns[i]->insert(filling_row[i]);
}
for (size_t i = 0; i < other_columns.size(); ++i)
other_columns[i]->insertDefault();
}
void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num)
{
for (size_t i = 0; i < source.size(); ++i)
dest[i]->insertFrom(*source[i], row_num);
}
}

View File

@ -0,0 +1,44 @@
#pragma once
#include <Core/SortDescription.h>
#include <Columns/IColumn.h>
#include <Common/FieldVisitors.h>
namespace DB
{
/// Compares fields in terms of sorting order, considering direction.
bool less(const Field & lhs, const Field & rhs, int direction);
bool equals(const Field & lhs, const Field & rhs);
/** Helps to implement modifier WITH FILL for ORDER BY clause.
* Stores row as array of fields and provides functions to generate next row for filling gaps and for comparing rows.
* Used in FillingBlockInputStream and in FillingTransform.
*/
class FillingRow
{
public:
FillingRow(const SortDescription & sort_description);
/// Generates next row according to fill 'from', 'to' and 'step' values.
bool next(const FillingRow & to_row);
void initFromDefaults(size_t from_pos = 0);
Field & operator[](size_t ind) { return row[ind]; }
const Field & operator[](size_t ind) const { return row[ind]; }
size_t size() const { return row.size(); }
bool operator<(const FillingRow & other) const;
bool operator==(const FillingRow & other) const;
int getDirection(size_t ind) const { return description[ind].direction; }
FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; }
private:
std::vector<Field> row;
SortDescription description;
};
void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row);
void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num);
}

View File

@ -155,7 +155,7 @@ BlockIO InterpreterInsertQuery::execute()
}
else if (query.data && !query.has_tail) /// can execute without additional data
{
res.in = std::make_shared<InputStreamFromASTInsertQuery>(query_ptr, nullptr, query_sample_block, context);
res.in = std::make_shared<InputStreamFromASTInsertQuery>(query_ptr, nullptr, query_sample_block, context, nullptr);
res.in = std::make_shared<NullAndDoCopyBlockInputStream>(res.in, res.out);
res.out = nullptr;
}

View File

@ -24,6 +24,7 @@
#include <DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h>
#include <DataStreams/ConvertingBlockInputStream.h>
#include <DataStreams/ReverseBlockInputStream.h>
#include <DataStreams/FillingBlockInputStream.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -44,6 +45,7 @@
#include <Interpreters/DatabaseAndTableWithAlias.h>
#include <Interpreters/JoinToSubqueryTransformVisitor.h>
#include <Interpreters/CrossToInnerJoinVisitor.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Storages/MergeTree/MergeTreeData.h>
#include <Storages/MergeTree/MergeTreeWhereOptimizer.h>
@ -57,6 +59,7 @@
#include <Core/Field.h>
#include <Core/Types.h>
#include <Columns/Collator.h>
#include <Common/FieldVisitors.h>
#include <Common/typeid_cast.h>
#include <Common/checkStackSize.h>
#include <Parsers/queryToString.h>
@ -81,6 +84,7 @@
#include <Processors/Transforms/CreatingSetsTransform.h>
#include <Processors/Transforms/RollupTransform.h>
#include <Processors/Transforms/CubeTransform.h>
#include <Processors/Transforms/FillingTransform.h>
#include <Processors/LimitTransform.h>
#include <Processors/Transforms/FinishSortingTransform.h>
#include <DataTypes/DataTypeAggregateFunction.h>
@ -103,6 +107,7 @@ namespace ErrorCodes
extern const int PARAMETER_OUT_OF_BOUND;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int INVALID_LIMIT_EXPRESSION;
extern const int INVALID_WITH_FILL_EXPRESSION;
}
namespace
@ -414,8 +419,6 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors()
Block InterpreterSelectQuery::getSampleBlockImpl()
{
FilterInfoPtr filter_info;
/// Need to create sets before analyzeExpressions(). Otherwise some sets for index won't be created.
query_analyzer->makeSetsForIndex(getSelectQuery().where());
query_analyzer->makeSetsForIndex(getSelectQuery().prewhere());
@ -427,8 +430,9 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
options.to_stage,
context,
storage,
true,
filter_info);
true, // only_types
{} // filter_info
);
if (options.to_stage == QueryProcessingStage::Enum::FetchColumns)
{
@ -682,8 +686,62 @@ InterpreterSelectQuery::analyzeExpressions(
return res;
}
static Field getWithFillFieldValue(const ASTPtr & node, const Context & context)
{
const auto & [field, type] = evaluateConstantExpression(node, context);
static SortDescription getSortDescription(const ASTSelectQuery & query)
if (!isColumnedAsNumber(type))
throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
return field;
}
static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, const Context & context)
{
FillColumnDescription descr;
if (order_by_elem.fill_from)
descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context);
if (order_by_elem.fill_to)
descr.fill_to = getWithFillFieldValue(order_by_elem.fill_to, context);
if (order_by_elem.fill_step)
descr.fill_step = getWithFillFieldValue(order_by_elem.fill_step, context);
else
descr.fill_step = order_by_elem.direction;
if (applyVisitor(FieldVisitorAccurateEquals(), descr.fill_step, Field{0}))
throw Exception("WITH FILL STEP value cannot be zero", ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (order_by_elem.direction == 1)
{
if (applyVisitor(FieldVisitorAccurateLess(), descr.fill_step, Field{0}))
throw Exception("WITH FILL STEP value cannot be negative for sorting in ascending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (!descr.fill_from.isNull() && !descr.fill_to.isNull() &&
applyVisitor(FieldVisitorAccurateLess(), descr.fill_to, descr.fill_from))
{
throw Exception("WITH FILL TO value cannot be less than FROM value for sorting in ascending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
}
else
{
if (applyVisitor(FieldVisitorAccurateLess(), Field{0}, descr.fill_step))
throw Exception("WITH FILL STEP value cannot be positive for sorting in descending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (!descr.fill_from.isNull() && !descr.fill_to.isNull() &&
applyVisitor(FieldVisitorAccurateLess(), descr.fill_from, descr.fill_to))
{
throw Exception("WITH FILL FROM value cannot be less than TO value for sorting in descending direction",
ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
}
return descr;
}
static SortDescription getSortDescription(const ASTSelectQuery & query, const Context & context)
{
SortDescription order_descr;
order_descr.reserve(query.orderBy()->children.size());
@ -696,13 +754,19 @@ static SortDescription getSortDescription(const ASTSelectQuery & query)
if (order_by_elem.collation)
collator = std::make_shared<Collator>(order_by_elem.collation->as<ASTLiteral &>().value.get<String>());
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator);
if (order_by_elem.with_fill)
{
FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context);
order_descr.emplace_back(name, order_by_elem.direction,
order_by_elem.nulls_direction, collator, true, fill_desc);
}
else
order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator);
}
return order_descr;
}
static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context)
{
const auto & [field, type] = evaluateConstantExpression(node, context);
@ -737,7 +801,7 @@ static std::pair<UInt64, UInt64> getLimitLengthAndOffset(const ASTSelectQuery &
static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context)
{
/// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY.
if (!query.distinct && !query.limitBy())
if (!query.distinct && !query.limitBy() && !query.limit_with_ties)
{
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
return limit_length + limit_offset;
@ -752,7 +816,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons
if (!merge_tree.hasSortingKey())
return {};
auto order_descr = getSortDescription(query);
auto order_descr = getSortDescription(query, context);
SortDescription prefix_order_descr;
int read_direction = order_descr.at(0).direction;
@ -927,6 +991,21 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
pipeline.streams.back() = std::make_shared<FilterBlockInputStream>(
pipeline.streams.back(), expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column);
// To remove additional columns in dry run
// For example, sample column which can be removed in this stage
if (expressions.prewhere_info->remove_columns_actions)
{
if constexpr (pipeline_with_processors)
{
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, expressions.prewhere_info->remove_columns_actions);
});
}
else
pipeline.streams.back() = std::make_shared<ExpressionBlockInputStream>(pipeline.streams.back(), expressions.prewhere_info->remove_columns_actions);
}
}
}
else
@ -1174,7 +1253,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
/** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT,
* limiting the number of rows in each up to `offset + limit`.
*/
if (query.limitLength() && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.has_limit_by && !settings.extremes)
if (query.limitLength() && !query.limit_with_ties && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.has_limit_by && !settings.extremes)
{
executePreLimit(pipeline);
}
@ -1207,6 +1286,8 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
executeLimitBy(pipeline);
}
executeWithFill(pipeline);
/** We must do projection after DISTINCT because projection may remove some columns.
*/
executeProjection(pipeline, expressions.final_projection);
@ -1223,7 +1304,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
executeSubqueriesInSetsAndJoins(pipeline, expressions.subqueries_for_sets);
}
template <typename TPipeline>
void InterpreterSelectQuery::executeFetchColumns(
QueryProcessingStage::Enum processing_stage, TPipeline & pipeline,
@ -1421,11 +1501,12 @@ void InterpreterSelectQuery::executeFetchColumns(
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size,
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size,
* then as the block size we will use limit + offset (not to read more from the table than requested),
* and also set the number of threads to 1.
*/
if (!query.distinct
&& !query.limit_with_ties
&& !query.prewhere()
&& !query.where()
&& !query.groupBy()
@ -1496,12 +1577,22 @@ void InterpreterSelectQuery::executeFetchColumns(
streams = {std::make_shared<NullBlockInputStream>(storage->getSampleBlockForColumns(required_columns))};
if (query_info.prewhere_info)
{
streams.back() = std::make_shared<FilterBlockInputStream>(
streams.back(),
prewhere_info->prewhere_actions,
prewhere_info->prewhere_column_name,
prewhere_info->remove_prewhere_column);
// To remove additional columns
// In some cases, we did not read any marks so that the pipeline.streams is empty
// Thus, some columns in prewhere are not removed as expected
// This leads to mismatched header in distributed table
if (query_info.prewhere_info->remove_columns_actions)
{
streams.back() = std::make_shared<ExpressionBlockInputStream>(streams.back(), query_info.prewhere_info->remove_columns_actions);
}
}
}
for (auto & stream : streams)
@ -2011,7 +2102,7 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E
void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info)
{
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query);
SortDescription order_descr = getSortDescription(query, context);
const Settings & settings = context.getSettingsRef();
UInt64 limit = getLimitForSorting(query, context);
@ -2083,7 +2174,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr so
void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info)
{
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query);
SortDescription order_descr = getSortDescription(query, context);
UInt64 limit = getLimitForSorting(query, context);
const Settings & settings = context.getSettingsRef();
@ -2164,7 +2255,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP
void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
{
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query);
SortDescription order_descr = getSortDescription(query, context);
UInt64 limit = getLimitForSorting(query, context);
const Settings & settings = context.getSettingsRef();
@ -2191,7 +2282,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query);
SortDescription order_descr = getSortDescription(query, context);
UInt64 limit = getLimitForSorting(query, context);
const Settings & settings = context.getSettingsRef();
@ -2238,7 +2329,7 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or
UInt64 limit_for_distinct = 0;
/// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows.
if (!query.orderBy() || !before_order)
if ((!query.orderBy() || !before_order) && !query.limit_with_ties)
limit_for_distinct = limit_length + limit_offset;
pipeline.transform([&](auto & stream)
@ -2307,9 +2398,16 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
if (query.limitLength())
{
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
SortDescription sort_descr;
if (query.limit_with_ties)
{
if (!query.orderBy())
throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
sort_descr = getSortDescription(query, context);
}
pipeline.transform([&, limit = limit_length + limit_offset](auto & stream)
{
stream = std::make_shared<LimitBlockInputStream>(stream, limit, 0, false);
stream = std::make_shared<LimitBlockInputStream>(stream, limit, 0, false, false, query.limit_with_ties, sort_descr);
});
}
}
@ -2421,17 +2519,73 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query))
always_read_till_end = true;
SortDescription order_descr;
if (query.limit_with_ties)
{
if (!query.orderBy())
throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
order_descr = getSortDescription(query, context);
}
UInt64 limit_length;
UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context);
pipeline.transform([&](auto & stream)
{
stream = std::make_shared<LimitBlockInputStream>(stream, limit_length, limit_offset, always_read_till_end);
stream = std::make_shared<LimitBlockInputStream>(stream, limit_length, limit_offset, always_read_till_end, false, query.limit_with_ties, order_descr);
});
}
}
void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline)
{
auto & query = getSelectQuery();
if (query.orderBy())
{
SortDescription order_descr = getSortDescription(query, context);
SortDescription fill_descr;
for (auto & desc : order_descr)
{
if (desc.with_fill)
fill_descr.push_back(desc);
}
if (fill_descr.empty())
return;
pipeline.transform([&](auto & stream)
{
stream = std::make_shared<FillingBlockInputStream>(stream, fill_descr);
});
}
}
void InterpreterSelectQuery::executeWithFill(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
if (query.orderBy())
{
SortDescription order_descr = getSortDescription(query, context);
SortDescription fill_descr;
for (auto & desc : order_descr)
{
if (desc.with_fill)
fill_descr.push_back(desc);
}
if (fill_descr.empty())
return;
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FillingTransform>(header, fill_descr);
});
}
}
void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
@ -2459,13 +2613,21 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline)
UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context);
SortDescription order_descr;
if (query.limit_with_ties)
{
if (!query.orderBy())
throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
order_descr = getSortDescription(query, context);
}
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type != QueryPipeline::StreamType::Main)
return nullptr;
return std::make_shared<LimitTransform>(
header, limit_length, limit_offset, always_read_till_end);
header, limit_length, limit_offset, always_read_till_end, query.limit_with_ties, order_descr);
});
}
}

View File

@ -204,6 +204,7 @@ private:
void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression);
void executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info);
void executeWithFill(Pipeline & pipeline);
void executeMergeSorted(Pipeline & pipeline);
void executePreLimit(Pipeline & pipeline);
void executeUnion(Pipeline & pipeline, Block header); /// If header is not empty, convert streams structure to it.
@ -221,6 +222,7 @@ private:
void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info);
void executeWithFill(QueryPipeline & pipeline);
void executeMergeSorted(QueryPipeline & pipeline);
void executePreLimit(QueryPipeline & pipeline);
void executeLimitBy(QueryPipeline & pipeline);

View File

@ -3,7 +3,6 @@
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Interpreters/Context.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTSelectQuery.h>

View File

@ -1,5 +1,4 @@
#include <Interpreters/SubqueryForSet.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/InterpreterSelectWithUnionQuery.h>
#include <DataStreams/LazyBlockInputStream.h>
@ -7,26 +6,14 @@ namespace DB
{
void SubqueryForSet::makeSource(std::shared_ptr<InterpreterSelectWithUnionQuery> & interpreter,
const std::unordered_map<String, String> & name_to_origin)
NamesWithAliases && joined_block_aliases_)
{
joined_block_aliases = std::move(joined_block_aliases_);
source = std::make_shared<LazyBlockInputStream>(interpreter->getSampleBlock(),
[interpreter]() mutable { return interpreter->execute().in; });
for (const auto & names : name_to_origin)
joined_block_aliases.emplace_back(names.second, names.first);
sample_block = source->getHeader();
for (const auto & name_with_alias : joined_block_aliases)
{
if (sample_block.has(name_with_alias.first))
{
auto pos = sample_block.getPositionByName(name_with_alias.first);
auto column = sample_block.getByPosition(pos);
sample_block.erase(pos);
column.name = name_with_alias.second;
sample_block.insert(std::move(column));
}
}
renameColumns(sample_block);
}
void SubqueryForSet::renameColumns(Block & block)

View File

@ -31,7 +31,7 @@ struct SubqueryForSet
StoragePtr table;
void makeSource(std::shared_ptr<InterpreterSelectWithUnionQuery> & interpreter,
const std::unordered_map<String, String> & name_to_origin);
NamesWithAliases && joined_block_aliases_);
Block renamedSampleBlock() const { return sample_block; }
void renameColumns(Block & block);

View File

@ -17,6 +17,7 @@
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
#include <Interpreters/RequiredSourceColumnsVisitor.h>
#include <Interpreters/GetAggregatesVisitor.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/ExpressionActions.h> /// getSmallestColumn()
#include <Parsers/ASTExpressionList.h>

View File

@ -1,7 +1,7 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <Interpreters/Aliases.h>
#include <Interpreters/AnalyzedJoin.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Storages/IStorage_fwd.h>
@ -11,6 +11,9 @@ namespace DB
NameSet removeDuplicateColumns(NamesAndTypesList & columns);
class ASTFunction;
class AnalyzedJoin;
class Context;
struct SelectQueryOptions;
struct SyntaxAnalyzerResult
{

View File

@ -21,6 +21,8 @@
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h>
#include <Storages/StorageInput.h>
#include <Interpreters/Quota.h>
#include <Interpreters/InterpreterFactory.h>
#include <Interpreters/ProcessList.h>
@ -31,6 +33,7 @@
#include <Common/ProfileEvents.h>
#include <Interpreters/DNSCacheUpdater.h>
#include <Common/SensitiveDataMasker.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
@ -76,7 +79,7 @@ static String prepareQueryForLogging(const String & query, Context & context)
// 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())
if (auto masker = SensitiveDataMasker::getInstance())
{
auto matches = masker->wipeSensitiveData(res);
if (matches > 0)
@ -178,7 +181,8 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
bool internal,
QueryProcessingStage::Enum stage,
bool has_query_tail,
bool allow_processors = true)
ReadBuffer * istr,
bool allow_processors)
{
time_t current_time = time(nullptr);
@ -273,6 +277,28 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Load external tables if they were provided
context.initializeExternalTablesIfSet();
auto * insert_query = ast->as<ASTInsertQuery>();
if (insert_query && insert_query->select)
{
/// Prepare Input storage before executing interpreter if we already got a buffer with data.
if (istr)
{
ASTPtr input_function;
insert_query->tryFindInputFunction(input_function);
if (input_function)
{
StoragePtr storage = context.executeTableFunction(input_function);
auto & input_storage = dynamic_cast<StorageInput &>(*storage);
BlockInputStreamPtr input_stream = std::make_shared<InputStreamFromASTInsertQuery>(ast, istr,
input_storage.getSampleBlock(), context, input_function);
input_storage.setInputStream(input_stream);
}
}
}
else
/// reset Input callbacks if query is not INSERT SELECT
context.resetInputCallbacks();
auto interpreter = InterpreterFactory::get(ast, context, stage);
bool use_processors = settings.experimental_use_processors && allow_processors && interpreter->canExecuteWithProcessors();
@ -527,7 +553,8 @@ BlockIO executeQuery(
bool allow_processors)
{
BlockIO streams;
std::tie(std::ignore, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context, internal, stage, !may_have_embedded_data, allow_processors);
std::tie(std::ignore, streams) = executeQueryImpl(query.data(), query.data() + query.size(), context,
internal, stage, !may_have_embedded_data, nullptr, allow_processors);
return streams;
}
@ -578,7 +605,7 @@ void executeQuery(
ASTPtr ast;
BlockIO streams;
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, may_have_tail);
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, may_have_tail, &istr, true);
auto & pipeline = streams.pipeline;
@ -586,7 +613,7 @@ void executeQuery(
{
if (streams.out)
{
InputStreamFromASTInsertQuery in(ast, &istr, streams.out->getHeader(), context);
InputStreamFromASTInsertQuery in(ast, &istr, streams.out->getHeader(), context, nullptr);
copyData(in, *streams.out);
}

View File

@ -18,6 +18,19 @@ namespace DB
std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns)
{
if (auto * expr = table_expression->as<ASTTableExpression>())
{
ASTPtr table;
if (expr->subquery)
table = expr->subquery;
else if (expr->table_function)
table = expr->table_function;
else if (expr->database_and_table_name)
table = expr->database_and_table_name;
return interpretSubquery(table, context, subquery_depth, required_source_columns);
}
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
const auto * subquery = table_expression->as<ASTSubquery>();
const auto * function = table_expression->as<ASTFunction>();

View File

@ -1,10 +1,17 @@
#include <iomanip>
#include <Parsers/ASTInsertQuery.h>
#include <Parsers/ASTFunction.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_USAGE_OF_INPUT;
}
void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const
{
frame.need_parens = false;
@ -50,4 +57,29 @@ void ASTInsertQuery::formatImpl(const FormatSettings & settings, FormatState & s
}
}
void tryFindInputFunctionImpl(const ASTPtr & ast, ASTPtr & input_function)
{
if (!ast)
return;
for (const auto & child : ast->children)
tryFindInputFunctionImpl(child, input_function);
if (const auto * table_function_ast = ast->as<ASTFunction>())
{
if (table_function_ast->name == "input")
{
if (input_function)
throw Exception("You can use 'input()' function only once per request.", ErrorCodes::INVALID_USAGE_OF_INPUT);
input_function = ast;
}
}
}
void ASTInsertQuery::tryFindInputFunction(ASTPtr & input_function) const
{
tryFindInputFunctionImpl(select, input_function);
}
}

View File

@ -30,6 +30,9 @@ public:
/// Query has additional data, which will be sent later
bool has_tail = false;
/// Try to find table function input() in SELECT part
void tryFindInputFunction(ASTPtr & input_function) const;
/** Get the text that identifies this element. */
String getID(char delim) const override { return "InsertQuery" + (delim + database) + delim + table; }

View File

@ -25,6 +25,26 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState
settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : "");
collation->formatImpl(settings, state, frame);
}
if (with_fill)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL " << (settings.hilite ? hilite_none : "");
if (fill_from)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : "");
fill_from->formatImpl(settings, state, frame);
}
if (fill_to)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : "");
fill_to->formatImpl(settings, state, frame);
}
if (fill_step)
{
settings.ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : "");
fill_step->formatImpl(settings, state, frame);
}
}
}
}

View File

@ -18,12 +18,22 @@ public:
/** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */
ASTPtr collation;
bool with_fill;
ASTPtr fill_from;
ASTPtr fill_to;
ASTPtr fill_step;
ASTOrderByElement(
const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_, ASTPtr & collation_)
const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_,
ASTPtr & collation_, const bool with_fill_, ASTPtr & fill_from_, ASTPtr & fill_to_, ASTPtr & fill_step_)
: direction(direction_)
, nulls_direction(nulls_direction_)
, nulls_direction_was_explicitly_specified(nulls_direction_was_explicitly_specified_)
, collation(collation_)
, with_fill(with_fill_)
, fill_from(fill_from_)
, fill_to(fill_to_)
, fill_step(fill_step_)
{
}

View File

@ -148,6 +148,8 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F
s.ostr << ", ";
}
limitLength()->formatImpl(s, state, frame);
if (limit_with_ties)
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << " WITH TIES" << (s.hilite ? hilite_none : "");
}
if (settings())

View File

@ -42,6 +42,7 @@ public:
bool group_by_with_totals = false;
bool group_by_with_rollup = false;
bool group_by_with_cube = false;
bool limit_with_ties = false;
ASTPtr & refSelect() { return getExpression(Expression::SELECT); }
ASTPtr & refTables() { return getExpression(Expression::TABLES); }

View File

@ -1360,7 +1360,12 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
ParserKeyword first("FIRST");
ParserKeyword last("LAST");
ParserKeyword collate("COLLATE");
ParserKeyword with_fill("WITH FILL");
ParserKeyword from("FROM");
ParserKeyword to("TO");
ParserKeyword step("STEP");
ParserStringLiteral collate_locale_parser;
ParserExpressionWithOptionalAlias exp_parser(false);
ASTPtr expr_elem;
if (!elem_p.parse(pos, expr_elem, expected))
@ -1395,7 +1400,27 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
return false;
}
node = std::make_shared<ASTOrderByElement>(direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node);
/// WITH FILL [FROM x] [TO y] [STEP z]
bool has_with_fill = false;
ASTPtr fill_from;
ASTPtr fill_to;
ASTPtr fill_step;
if (with_fill.ignore(pos))
{
has_with_fill = true;
if (from.ignore(pos) && !exp_parser.parse(pos, fill_from, expected))
return false;
if (to.ignore(pos) && !exp_parser.parse(pos, fill_to, expected))
return false;
if (step.ignore(pos) && !exp_parser.parse(pos, fill_step, expected))
return false;
}
node = std::make_shared<ASTOrderByElement>(
direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node,
has_with_fill, fill_from, fill_to, fill_step);
node->children.push_back(expr_elem);
if (locale_node)
node->children.push_back(locale_node);

View File

@ -289,6 +289,7 @@ protected:
/** Element of ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified
* and optionally, NULLS LAST|FIRST
* and optionally, COLLATE 'locale'.
* and optionally, WITH FILL [FROM x] [TO y] [STEP z]
*/
class ParserOrderByElement : public IParserBase
{

View File

@ -8,7 +8,7 @@
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserInsertQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTFunction.h>
#include <Common/typeid_cast.h>
namespace DB
@ -97,6 +97,10 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
pos = before_select;
ParserSelectWithUnionQuery select_p;
select_p.parse(pos, select, expected);
/// FORMAT section is expected if we have input() in SELECT part
if (s_format.ignore(pos, expected) && !name_p.parse(pos, format, expected))
return false;
}
else
{

View File

@ -17,6 +17,7 @@ namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
extern const int TOP_AND_LIMIT_TOGETHER;
extern const int WITH_TIES_WITHOUT_ORDER_BY;
}
@ -41,6 +42,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_rollup("ROLLUP");
ParserKeyword s_cube("CUBE");
ParserKeyword s_top("TOP");
ParserKeyword s_with_ties("WITH TIES");
ParserKeyword s_offset("OFFSET");
ParserNotEmptyExpressionList exp_list(false);
@ -76,7 +78,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
}
/// SELECT [DISTINCT] [TOP N] expr list
/// SELECT [DISTINCT] [TOP N [WITH TIES]] expr list
{
if (!s_select.ignore(pos, expected))
return false;
@ -100,6 +102,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!num.parse(pos, limit_length, expected))
return false;
}
if (s_with_ties.ignore(pos, expected))
select_query->limit_with_ties = true;
}
if (!exp_list_for_select_clause.parse(pos, select_expression_list, expected))
@ -197,12 +202,18 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
limit_offset = limit_length;
if (!exp_elem.parse(pos, limit_length, expected))
return false;
if (s_with_ties.ignore(pos, expected))
select_query->limit_with_ties = true;
}
else if (s_offset.ignore(pos, expected))
{
if (!exp_elem.parse(pos, limit_offset, expected))
return false;
}
else if (s_with_ties.ignore(pos, expected))
select_query->limit_with_ties = true;
if (s_by.ignore(pos, expected))
{
limit_by_length = limit_length;
@ -215,7 +226,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
}
}
/// LIMIT length | LIMIT offset, length
/// LIMIT length [WITH TIES] | LIMIT offset, length [WITH TIES]
if (s_limit.ignore(pos, expected))
{
if (!limit_by_length|| limit_length)
@ -237,8 +248,15 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (!exp_elem.parse(pos, limit_offset, expected))
return false;
}
if (s_with_ties.ignore(pos, expected))
select_query->limit_with_ties = true;
}
/// WITH TIES was used without ORDER BY
if (!order_expression_list && select_query->limit_with_ties)
throw Exception("Can not use WITH TIES without ORDER BY", ErrorCodes::WITH_TIES_WITHOUT_ORDER_BY);
/// SETTINGS key1 = value1, key2 = value2, ...
if (s_settings.ignore(pos, expected))
{

View File

@ -6,19 +6,26 @@ namespace DB
LimitTransform::LimitTransform(
const Block & header_, size_t limit_, size_t offset_,
bool always_read_till_end_)
bool always_read_till_end_, bool with_ties_,
const SortDescription & description_)
: IProcessor({header_}, {header_})
, input(inputs.front()), output(outputs.front())
, limit(limit_), offset(offset_)
, always_read_till_end(always_read_till_end_)
, with_ties(with_ties_), description(description_)
{
for (const auto & desc : description)
{
if (!desc.column_name.empty())
sort_column_positions.push_back(header_.getPositionByName(desc.column_name));
else
sort_column_positions.push_back(desc.column_number);
}
}
LimitTransform::Status LimitTransform::prepare()
{
/// Check can output.
bool output_finished = false;
if (output.isFinished())
@ -46,7 +53,7 @@ LimitTransform::Status LimitTransform::prepare()
}
/// Check if we are done with pushing.
bool pushing_is_finished = rows_read >= offset + limit;
bool pushing_is_finished = (rows_read >= offset + limit) && ties_row_ref.empty();
if (pushing_is_finished)
{
if (!always_read_till_end)
@ -116,6 +123,13 @@ LimitTransform::Status LimitTransform::prepare()
if (output.hasData())
return Status::PortFull;
if (with_ties && rows_read == offset + limit)
{
SharedChunkPtr shared_chunk = new detail::SharedChunk(current_chunk.clone());
shared_chunk->sort_columns = extractSortColumns(shared_chunk->getColumns());
ties_row_ref.set(shared_chunk, &shared_chunk->sort_columns, shared_chunk->getNumRows() - 1);
}
output.push(std::move(current_chunk));
has_block = false;
@ -132,8 +146,39 @@ LimitTransform::Status LimitTransform::prepare()
void LimitTransform::work()
{
size_t num_rows = current_chunk.getNumRows();
size_t num_columns = current_chunk.getNumColumns();
SharedChunkPtr shared_chunk = new detail::SharedChunk(std::move(current_chunk));
shared_chunk->sort_columns = extractSortColumns(shared_chunk->getColumns());
size_t num_rows = shared_chunk->getNumRows();
size_t num_columns = shared_chunk->getNumColumns();
if (!ties_row_ref.empty() && rows_read >= offset + limit)
{
UInt64 len;
for (len = 0; len < num_rows; ++len)
{
SharedChunkRowRef current_row;
current_row.set(shared_chunk, &shared_chunk->sort_columns, len);
if (current_row != ties_row_ref)
{
ties_row_ref.reset();
break;
}
}
auto columns = shared_chunk->detachColumns();
if (len < num_rows)
{
for (size_t i = 0; i < num_columns; ++i)
columns[i] = columns[i]->cut(0, len);
}
current_chunk.setColumns(std::move(columns), len);
block_processed = true;
return;
}
/// return a piece of the block
size_t start = std::max(
@ -145,7 +190,33 @@ void LimitTransform::work()
static_cast<Int64>(rows_read) - static_cast<Int64>(offset),
static_cast<Int64>(limit) + static_cast<Int64>(offset) - static_cast<Int64>(rows_read) + static_cast<Int64>(num_rows)));
auto columns = current_chunk.detachColumns();
/// check if other rows in current block equals to last one in limit
if (with_ties)
{
ties_row_ref.set(shared_chunk, &shared_chunk->sort_columns, start + length - 1);
SharedChunkRowRef current_row;
for (size_t i = ties_row_ref.row_num + 1; i < num_rows; ++i)
{
current_row.set(shared_chunk, &shared_chunk->sort_columns, i);
if (current_row == ties_row_ref)
++length;
else
{
ties_row_ref.reset();
break;
}
}
}
if (length == num_rows)
{
current_chunk = std::move(*shared_chunk);
block_processed = true;
return;
}
auto columns = shared_chunk->detachColumns();
for (size_t i = 0; i < num_columns; ++i)
columns[i] = columns[i]->cut(start, length);
@ -155,5 +226,15 @@ void LimitTransform::work()
block_processed = true;
}
ColumnRawPtrs LimitTransform::extractSortColumns(const Columns & columns)
{
ColumnRawPtrs res;
res.reserve(description.size());
for (size_t pos : sort_column_positions)
res.push_back(columns[pos].get());
return res;
}
}

View File

@ -1,7 +1,8 @@
#pragma once
#include <Processors/IProcessor.h>
#include <Processors/SharedChunk.h>
#include <Core/SortDescription.h>
namespace DB
{
@ -23,10 +24,18 @@ private:
UInt64 rows_before_limit_at_least = 0;
bool with_ties;
const SortDescription description;
SharedChunkRowRef ties_row_ref;
std::vector<size_t> sort_column_positions;
ColumnRawPtrs extractSortColumns(const Columns & columns);
public:
LimitTransform(
const Block & header_, size_t limit_, size_t offset_,
bool always_read_till_end_ = false);
bool always_read_till_end_ = false, bool with_ties_ = false,
const SortDescription & description_ = {});
String getName() const override { return "Limit"; }

View File

@ -0,0 +1,91 @@
#pragma once
#include <algorithm>
#include <Processors/Chunk.h>
#include <Columns/IColumn.h>
#include <boost/smart_ptr/intrusive_ptr.hpp>
namespace DB
{
/// Allows you refer to the row in the block and hold the block ownership,
/// and thus avoid creating a temporary row object.
/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`;
/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece;
/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr;
/// The reference counter is not atomic, since it is used from one thread.
namespace detail
{
struct SharedChunk : Chunk
{
int refcount = 0;
ColumnRawPtrs all_columns;
ColumnRawPtrs sort_columns;
SharedChunk(Chunk && chunk) : Chunk(std::move(chunk)) {}
};
}
inline void intrusive_ptr_add_ref(detail::SharedChunk * ptr)
{
++ptr->refcount;
}
inline void intrusive_ptr_release(detail::SharedChunk * ptr)
{
if (0 == --ptr->refcount)
delete ptr;
}
using SharedChunkPtr = boost::intrusive_ptr<detail::SharedChunk>;
struct SharedChunkRowRef
{
ColumnRawPtrs * columns = nullptr;
size_t row_num;
SharedChunkPtr shared_block;
void swap(SharedChunkRowRef & other)
{
std::swap(columns, other.columns);
std::swap(row_num, other.row_num);
std::swap(shared_block, other.shared_block);
}
/// The number and types of columns must match.
bool operator==(const SharedChunkRowRef & other) const
{
size_t size = columns->size();
for (size_t i = 0; i < size; ++i)
if (0 != (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1))
return false;
return true;
}
bool operator!=(const SharedChunkRowRef & other) const
{
return !(*this == other);
}
void reset()
{
SharedChunkRowRef empty;
swap(empty);
}
bool empty() const { return columns == nullptr; }
size_t size() const { return empty() ? 0 : columns->size(); }
void set(SharedChunkPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_)
{
shared_block = shared_block_;
columns = columns_;
row_num = row_num_;
}
};
}

View File

@ -0,0 +1,201 @@
#include <Processors/Transforms/FillingTransform.h>
#include <Interpreters/convertFieldToType.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INVALID_WITH_FILL_EXPRESSION;
}
FillingTransform::FillingTransform(
const Block & header_, const SortDescription & sort_description_)
: ISimpleTransform(header_, header_, true)
, sort_description(sort_description_)
, filling_row(sort_description_)
, next_row(sort_description_)
{
std::vector<bool> is_fill_column(header_.columns());
for (const auto & elem : sort_description)
is_fill_column[header_.getPositionByName(elem.column_name)] = true;
auto try_convert_fields = [](FillColumnDescription & descr, const DataTypePtr & type)
{
auto max_type = Field::Types::Null;
WhichDataType which(type);
DataTypePtr to_type;
if (isInteger(type) || which.isDateOrDateTime())
{
max_type = Field::Types::Int64;
to_type = std::make_shared<DataTypeInt64>();
}
else if (which.isFloat())
{
max_type = Field::Types::Float64;
to_type = std::make_shared<DataTypeFloat64>();
}
if (descr.fill_from.getType() > max_type || descr.fill_to.getType() > max_type
|| descr.fill_step.getType() > max_type)
return false;
descr.fill_from = convertFieldToType(descr.fill_from, *to_type);
descr.fill_to = convertFieldToType(descr.fill_to, *to_type);
descr.fill_step = convertFieldToType(descr.fill_step, *to_type);
return true;
};
for (size_t i = 0; i < header_.columns(); ++i)
{
if (is_fill_column[i])
{
size_t pos = fill_column_positions.size();
auto & descr = filling_row.getFillDescription(pos);
auto type = header_.getByPosition(i).type;
if (!try_convert_fields(descr, type))
throw Exception("Incompatible types of WITH FILL expression values with column type "
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
if (type->isValueRepresentedByUnsignedInteger() &&
((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) ||
(!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1))))
{
throw Exception("WITH FILL bound values cannot be negative for unsigned type "
+ type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION);
}
fill_column_positions.push_back(i);
}
else
other_column_positions.push_back(i);
}
}
IProcessor::Status FillingTransform::prepare()
{
if (input.isFinished() && !output.isFinished() && !has_input && !generate_suffix)
{
should_insert_first = next_row < filling_row;
for (size_t i = 0; i < filling_row.size(); ++i)
next_row[i] = filling_row.getFillDescription(i).fill_to;
if (filling_row < next_row)
{
generate_suffix = true;
return Status::Ready;
}
}
return ISimpleTransform::prepare();
}
void FillingTransform::transform(Chunk & chunk)
{
Columns old_fill_columns;
Columns old_other_columns;
MutableColumns res_fill_columns;
MutableColumns res_other_columns;
auto init_columns_by_positions = [](const Columns & old_columns, Columns & new_columns,
MutableColumns & new_mutable_columns, const Positions & positions)
{
for (size_t pos : positions)
{
new_columns.push_back(old_columns[pos]);
new_mutable_columns.push_back(old_columns[pos]->cloneEmpty()->assumeMutable());
}
};
if (generate_suffix)
{
const auto & empty_columns = inputs.front().getHeader().getColumns();
init_columns_by_positions(empty_columns, old_fill_columns, res_fill_columns, fill_column_positions);
init_columns_by_positions(empty_columns, old_other_columns, res_other_columns, other_column_positions);
if (should_insert_first && filling_row < next_row)
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
while (filling_row.next(next_row))
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
setResultColumns(chunk, res_fill_columns, res_other_columns);
return;
}
size_t num_rows = chunk.getNumRows();
auto old_columns = chunk.detachColumns();
init_columns_by_positions(old_columns, old_fill_columns, res_fill_columns, fill_column_positions);
init_columns_by_positions(old_columns, old_other_columns, res_other_columns, other_column_positions);
if (first)
{
for (size_t i = 0; i < filling_row.size(); ++i)
{
auto current_value = (*old_fill_columns[i])[0];
const auto & fill_from = filling_row.getFillDescription(i).fill_from;
if (!fill_from.isNull() && !equals(current_value, fill_from))
{
filling_row.initFromDefaults(i);
if (less(fill_from, current_value, filling_row.getDirection(i)))
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
break;
}
filling_row[i] = current_value;
}
first = false;
}
for (size_t row_ind = 0; row_ind < num_rows; ++row_ind)
{
should_insert_first = next_row < filling_row;
for (size_t i = 0; i < filling_row.size(); ++i)
{
auto current_value = (*old_fill_columns[i])[row_ind];
const auto & fill_to = filling_row.getFillDescription(i).fill_to;
if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i)))
next_row[i] = current_value;
else
next_row[i] = fill_to;
}
/// A case, when at previous step row was initialized from defaults 'fill_from' values
/// and probably we need to insert it to block.
if (should_insert_first && filling_row < next_row)
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
/// Insert generated filling row to block, while it is less than current row in block.
while (filling_row.next(next_row))
insertFromFillingRow(res_fill_columns, res_other_columns, filling_row);
copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind);
copyRowFromColumns(res_other_columns, old_other_columns, row_ind);
}
setResultColumns(chunk, res_fill_columns, res_other_columns);
}
void FillingTransform::setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const
{
MutableColumns result_columns(fill_columns.size() + other_columns.size());
/// fill_columns always non-empty.
size_t num_rows = fill_columns[0]->size();
for (size_t i = 0; i < fill_columns.size(); ++i)
result_columns[fill_column_positions[i]] = std::move(fill_columns[i]);
for (size_t i = 0; i < other_columns.size(); ++i)
result_columns[other_column_positions[i]] = std::move(other_columns[i]);
chunk.setColumns(std::move(result_columns), num_rows);
}
}

View File

@ -0,0 +1,42 @@
#pragma once
#include <Processors/ISimpleTransform.h>
#include <Core/SortDescription.h>
#include <Interpreters/FillingRow.h>
namespace DB
{
/** Implements modifier WITH FILL of ORDER BY clause.
* It fills gaps in data stream by rows with missing values in columns with set WITH FILL and deafult values in other columns.
* Optionally FROM, TO and STEP values can be specified.
*/
class FillingTransform : public ISimpleTransform
{
public:
FillingTransform(const Block & header_, const SortDescription & fill_description_);
String getName() const override { return "FillingTransform"; }
Status prepare() override;
protected:
void transform(Chunk & Chunk) override;
private:
void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const;
const SortDescription sort_description; /// Contains only rows with WITH FILL.
FillingRow filling_row; /// Current row, which is used to fill gaps.
FillingRow next_row; /// Row to which we need to generate filling rows.
using Positions = std::vector<size_t>;
Positions fill_column_positions;
Positions other_column_positions;
bool first = true;
bool generate_suffix = false;
/// Determines should we insert filling row before start generating next rows.
bool should_insert_first = false;
};
}

View File

@ -2,46 +2,13 @@
#include <Processors/IProcessor.h>
#include <Core/SortDescription.h>
#include <Core/SortCursor.h>
#include <Processors/SharedChunk.h>
#include <queue>
namespace DB
{
/// Allows you refer to the row in the block and hold the block ownership,
/// and thus avoid creating a temporary row object.
/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`;
/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece;
/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr;
/// The reference counter is not atomic, since it is used from one thread.
namespace detail
{
struct SharedChunk : Chunk
{
int refcount = 0;
ColumnRawPtrs all_columns;
ColumnRawPtrs sort_columns;
SharedChunk(Chunk && chunk) : Chunk(std::move(chunk)) {}
};
}
using SharedChunkPtr = boost::intrusive_ptr<detail::SharedChunk>;
inline void intrusive_ptr_add_ref(detail::SharedChunk * ptr)
{
++ptr->refcount;
}
inline void intrusive_ptr_release(detail::SharedChunk * ptr)
{
if (0 == --ptr->refcount)
delete ptr;
}
class MergingSortedTransform : public IProcessor
{
public:

View File

@ -189,7 +189,9 @@ ConnectionPoolPtr StorageDistributedDirectoryMonitor::createPool(const std::stri
auto pools = createPoolsForAddresses(name, pool_factory);
return pools.size() == 1 ? pools.front() : std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM);
const auto settings = storage.global_context.getSettings();
return pools.size() == 1 ? pools.front() : std::make_shared<ConnectionPoolWithFailover>(pools, LoadBalancing::RANDOM,
settings.distributed_replica_error_half_life.totalSeconds(), settings.distributed_replica_error_cap);
}

View File

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

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