mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 07:31:57 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
895bf554aa
@ -1,6 +1,6 @@
|
||||
#This strings autochanged from release_lib.sh :
|
||||
set(VERSION_DESCRIBE v1.1.54207-testing)
|
||||
set(VERSION_REVISION 54207)
|
||||
set(VERSION_DESCRIBE v1.1.54208-testing)
|
||||
set(VERSION_REVISION 54208)
|
||||
#===end of autochange
|
||||
|
||||
set (VERSION_MAJOR 1)
|
||||
|
@ -1,3 +1,3 @@
|
||||
<!-- Конфиг, который ищется в текущей директории. -->
|
||||
<!-- Config, which is searched in the current directory -->
|
||||
<config>
|
||||
</config>
|
||||
|
@ -203,7 +203,6 @@ void RemoteBlockInputStream::readSuffixImpl()
|
||||
/** If one of:
|
||||
* - nothing started to do;
|
||||
* - received all packets before EndOfStream;
|
||||
* - получили с одной реплики эксепшен;
|
||||
* - received exception from one replica;
|
||||
* - received an unknown packet from one replica;
|
||||
* then you do not need to read anything.
|
||||
|
@ -219,7 +219,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
else
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
||||
/// Список типов агрегатных функций.
|
||||
/// List of types of aggregate functions.
|
||||
std::stringstream aggregate_functions_typenames_str;
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
{
|
||||
@ -248,14 +248,14 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
|
||||
auto get_code = [method_typename, method_typename_two_level, aggregate_functions_typenames]
|
||||
{
|
||||
/// Короткий кусок кода, представляющий собой явное инстанцирование шаблона.
|
||||
/// A short piece of code, which is an explicit instantiation of the template.
|
||||
std::stringstream code;
|
||||
code << /// Нет явного включения заголовочного файла. Он подключается с помощью опции компилятора -include.
|
||||
code << /// No explicit inclusion of the header file. It is included using the -include compiler option.
|
||||
"namespace DB\n"
|
||||
"{\n"
|
||||
"\n";
|
||||
|
||||
/// Может быть до двух инстанцирований шаблона - для обычного и two_level вариантов.
|
||||
/// There can be up to two instantiations for the template - for normal and two_level options.
|
||||
auto append_code_for_specialization =
|
||||
[&code, &aggregate_functions_typenames] (const std::string & method_typename, const std::string & suffix)
|
||||
{
|
||||
@ -283,7 +283,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
"}\n"
|
||||
"\n"
|
||||
"void * getPtr" << suffix << "() __attribute__((__visibility__(\"default\")));\n"
|
||||
"void * getPtr" << suffix << "()\n" /// Без этой обёртки непонятно, как достать нужный символ из скомпилированной библиотеки.
|
||||
"void * getPtr" << suffix << "()\n" /// Without this wrapper, it's not clear how to get the desired symbol from the compiled library.
|
||||
"{\n"
|
||||
"\treturn reinterpret_cast<void *>(&wrapper" << suffix << ");\n"
|
||||
"}\n";
|
||||
@ -293,7 +293,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
append_code_for_specialization(method_typename, "");
|
||||
else
|
||||
{
|
||||
/// Для метода without_key.
|
||||
/// For `without_key` method.
|
||||
code <<
|
||||
"template void Aggregator::executeSpecializedWithoutKey<\n"
|
||||
"\t" << "TypeList<" << aggregate_functions_typenames << ">>(\n"
|
||||
@ -322,7 +322,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
append_code_for_specialization(method_typename_two_level, "TwoLevel");
|
||||
else
|
||||
{
|
||||
/// Заглушка.
|
||||
/// The stub.
|
||||
code <<
|
||||
"void * getPtrTwoLevel() __attribute__((__visibility__(\"default\")));\n"
|
||||
"void * getPtrTwoLevel()\n"
|
||||
@ -340,7 +340,7 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
auto compiled_data_owned_by_callback = compiled_data;
|
||||
auto on_ready = [compiled_data_owned_by_callback] (SharedLibraryPtr & lib)
|
||||
{
|
||||
if (compiled_data_owned_by_callback.unique()) /// Aggregator уже уничтожен.
|
||||
if (compiled_data_owned_by_callback.unique()) /// Aggregator is already destroyed.
|
||||
return;
|
||||
|
||||
compiled_data_owned_by_callback->compiled_aggregator = lib;
|
||||
@ -348,16 +348,16 @@ void Aggregator::compileIfPossible(AggregatedDataVariants::Type type)
|
||||
compiled_data_owned_by_callback->compiled_two_level_method_ptr = lib->get<void * (*) ()>("_ZN2DB14getPtrTwoLevelEv")();
|
||||
};
|
||||
|
||||
/** Если библиотека уже была скомпилирована, то возвращается ненулевой SharedLibraryPtr.
|
||||
* Если библиотека не была скомпилирована, то увеличивается счётчик, и возвращается nullptr.
|
||||
* Если счётчик достигнул значения min_count_to_compile, то асинхронно (в отдельном потоке) запускается компиляция,
|
||||
* по окончании которой вызывается колбэк on_ready.
|
||||
/** If the library has already been compiled, a non-zero SharedLibraryPtr is returned.
|
||||
* If the library was not compiled, then the counter is incremented, and nullptr is returned.
|
||||
* If the counter has reached the value min_count_to_compile, then the compilation starts asynchronously (in a separate thread)
|
||||
* at the end of which `on_ready` callback is called.
|
||||
*/
|
||||
SharedLibraryPtr lib = params.compiler->getOrCount(key, params.min_count_to_compile,
|
||||
"-include " INTERNAL_COMPILER_HEADERS "/dbms/src/Interpreters/SpecializedAggregator.h",
|
||||
get_code, on_ready);
|
||||
|
||||
/// Если результат уже готов.
|
||||
/// If the result is already ready.
|
||||
if (lib)
|
||||
on_ready(lib);
|
||||
}
|
||||
@ -527,9 +527,9 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const
|
||||
{
|
||||
try
|
||||
{
|
||||
/** Может возникнуть исключение при нехватке памяти.
|
||||
* Для того, чтобы потом всё правильно уничтожилось, "откатываем" часть созданных состояний.
|
||||
* Код не очень удобный.
|
||||
/** An exception may occur if there is a shortage of memory.
|
||||
* In order that then everything is properly destroyed, we "roll back" some of the created states.
|
||||
* The code is not very convenient.
|
||||
*/
|
||||
aggregate_functions[j]->create(aggregate_data + offsets_of_aggregate_states[j]);
|
||||
}
|
||||
@ -544,9 +544,9 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const
|
||||
}
|
||||
|
||||
|
||||
/** Интересно - если убрать noinline, то gcc зачем-то инлайнит эту функцию, и производительность уменьшается (~10%).
|
||||
* (Возможно из-за того, что после инлайна этой функции, перестают инлайниться более внутренние функции.)
|
||||
* Инлайнить не имеет смысла, так как внутренний цикл находится целиком внутри этой функции.
|
||||
/** It's interesting - if you remove `noinline`, then gcc for some reason will inline this function, and the performance decreases (~ 10%).
|
||||
* (Probably because after the inline of this function, more internal functions no longer be inlined.)
|
||||
* Inline does not make sense, since the inner loop is entirely inside this function.
|
||||
*/
|
||||
template <typename Method>
|
||||
void NO_INLINE Aggregator::executeImpl(
|
||||
@ -586,27 +586,27 @@ void NO_INLINE Aggregator::executeImplCase(
|
||||
StringRefs & keys,
|
||||
AggregateDataPtr overflow_row) const
|
||||
{
|
||||
/// NOTE При редактировании этого кода, обратите также внимание на SpecializedAggregator.h.
|
||||
/// NOTE When editing this code, also pay attention to SpecializedAggregator.h.
|
||||
|
||||
/// Для всех строчек.
|
||||
/// For all rows.
|
||||
typename Method::iterator it;
|
||||
typename Method::Key prev_key;
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
|
||||
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
|
||||
bool inserted; /// Inserted a new key, or was this key already?
|
||||
bool overflow = false; /// The new key did not fit in the hash table because of no_more_keys.
|
||||
|
||||
/// Получаем ключ для вставки в хэш-таблицу.
|
||||
/// Get the key to insert into the hash table.
|
||||
typename Method::Key key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
|
||||
|
||||
if (!no_more_keys) /// Вставляем.
|
||||
if (!no_more_keys) /// Insert.
|
||||
{
|
||||
/// Оптимизация для часто повторяющихся ключей.
|
||||
/// Optimization for frequently duplicating keys.
|
||||
if (!Method::no_consecutive_keys_optimization)
|
||||
{
|
||||
if (i != 0 && key == prev_key)
|
||||
{
|
||||
/// Добавляем значения в агрегатные функции.
|
||||
/// Add values to the aggregate functions.
|
||||
AggregateDataPtr value = Method::getAggregateData(it->second);
|
||||
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
||||
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool);
|
||||
@ -622,26 +622,26 @@ void NO_INLINE Aggregator::executeImplCase(
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Будем добавлять только если ключ уже есть.
|
||||
/// Add only if the key already exists.
|
||||
inserted = false;
|
||||
it = method.data.find(key);
|
||||
if (method.data.end() == it)
|
||||
overflow = true;
|
||||
}
|
||||
|
||||
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
|
||||
/// If the key does not fit, and the data does not need to be aggregated in a separate row, then there's nothing to do.
|
||||
if (no_more_keys && overflow && !overflow_row)
|
||||
{
|
||||
method.onExistingKey(key, keys, *aggregates_pool);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
|
||||
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
|
||||
if (inserted)
|
||||
{
|
||||
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
|
||||
|
||||
/// exception-safety - если не удалось выделить память или создать состояния, то не будут вызываться деструкторы.
|
||||
/// exception-safety - if you can not allocate memory or create states, then destructors will not be called.
|
||||
aggregate_data = nullptr;
|
||||
|
||||
method.onNewKey(*it, params.keys_size, i, keys, *aggregates_pool);
|
||||
@ -655,7 +655,7 @@ void NO_INLINE Aggregator::executeImplCase(
|
||||
|
||||
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
|
||||
|
||||
/// Добавляем значения в агрегатные функции.
|
||||
/// Add values to the aggregate functions.
|
||||
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
||||
(*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool);
|
||||
}
|
||||
@ -671,7 +671,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
|
||||
AggregateFunctionInstruction * aggregate_instructions,
|
||||
Arena * arena) const
|
||||
{
|
||||
/// Оптимизация в случае единственной агрегатной функции count.
|
||||
/// Optimization in the case of a single aggregate function `count`.
|
||||
AggregateFunctionCount * agg_count = params.aggregates_size == 1
|
||||
? typeid_cast<AggregateFunctionCount *>(aggregate_functions[0])
|
||||
: NULL;
|
||||
@ -682,7 +682,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl(
|
||||
{
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
/// Добавляем значения
|
||||
/// Adding values
|
||||
for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst)
|
||||
(*inst->func)(inst->that, res + inst->state_offset, inst->arguments, i, arena);
|
||||
}
|
||||
@ -700,18 +700,18 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
if (isCancelled())
|
||||
return true;
|
||||
|
||||
/// result будет уничтожать состояния агрегатных функций в деструкторе
|
||||
/// `result` will destroy the states of aggregate functions in the destructor
|
||||
result.aggregator = this;
|
||||
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i].resize(params.aggregates[i].arguments.size());
|
||||
|
||||
/** Константные столбцы не поддерживаются напрямую при агрегации.
|
||||
* Чтобы они всё-равно работали, материализуем их.
|
||||
/** Constant columns are not supported directly during aggregation.
|
||||
* To make them work anyway, we materialize them.
|
||||
*/
|
||||
Columns materialized_columns;
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = block.safeGetByPosition(params.keys[i]).column.get();
|
||||
@ -750,7 +750,7 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
|
||||
size_t rows = block.rows();
|
||||
|
||||
/// Каким способом выполнять агрегацию?
|
||||
/// How to perform the aggregation?
|
||||
if (result.empty())
|
||||
{
|
||||
result.init(chooseAggregationMethod(key_columns, key_sizes));
|
||||
@ -772,12 +772,12 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
result.without_key = place;
|
||||
}
|
||||
|
||||
/// Выбираем один из методов агрегации и вызываем его.
|
||||
/// We select one of the aggregation methods and call it.
|
||||
|
||||
/// Для случая, когда нет ключей (всё агегировать в одну строку).
|
||||
/// For the case when there are no keys (all aggregate into one row).
|
||||
if (result.type == AggregatedDataVariants::Type::without_key)
|
||||
{
|
||||
/// Если есть динамически скомпилированный код.
|
||||
/// If there is a dynamically compiled code.
|
||||
if (compiled_data->compiled_method_ptr)
|
||||
{
|
||||
reinterpret_cast<
|
||||
@ -789,12 +789,12 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Сюда пишутся данные, не поместившиеся в max_rows_to_group_by при group_by_overflow_mode = any.
|
||||
/// This is where data is written that does not fit in `max_rows_to_group_by` with `group_by_overflow_mode = any`.
|
||||
AggregateDataPtr overflow_row_ptr = params.overflow_row ? result.without_key : nullptr;
|
||||
|
||||
bool is_two_level = result.isTwoLevel();
|
||||
|
||||
/// Скомпилированный код, для обычной структуры.
|
||||
/// Compiled code, for the normal structure.
|
||||
if (!is_two_level && compiled_data->compiled_method_ptr)
|
||||
{
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
@ -810,7 +810,7 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
APPLY_FOR_AGGREGATED_VARIANTS(M)
|
||||
#undef M
|
||||
}
|
||||
/// Скомпилированный код, для two-level структуры.
|
||||
/// Compiled code, for a two-level structure.
|
||||
else if (is_two_level && compiled_data->compiled_two_level_method_ptr)
|
||||
{
|
||||
#define M(NAME) \
|
||||
@ -826,7 +826,7 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
APPLY_FOR_VARIANTS_TWO_LEVEL(M)
|
||||
#undef M
|
||||
}
|
||||
/// Когда нет динамически скомпилированного кода.
|
||||
/// When there is no dynamically compiled code.
|
||||
else
|
||||
{
|
||||
#define M(NAME, IS_TWO_LEVEL) \
|
||||
@ -845,24 +845,24 @@ bool Aggregator::executeOnBlock(Block & block, AggregatedDataVariants & result,
|
||||
if (current_memory_tracker)
|
||||
current_memory_usage = current_memory_tracker->get();
|
||||
|
||||
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Здесь учитываются все результаты в сумме, из разных потоков.
|
||||
auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Here all the results in the sum are taken into account, from different threads.
|
||||
|
||||
bool worth_convert_to_two_level
|
||||
= (params.group_by_two_level_threshold && result_size >= params.group_by_two_level_threshold)
|
||||
|| (params.group_by_two_level_threshold_bytes && result_size_bytes >= static_cast<Int64>(params.group_by_two_level_threshold_bytes));
|
||||
|
||||
/** Преобразование в двухуровневую структуру данных.
|
||||
* Она позволяет делать, в последующем, эффективный мердж - либо экономный по памяти, либо распараллеленный.
|
||||
/** Converting to a two-level data structure.
|
||||
* It allows you to make, in the subsequent, an effective merge - either economical from memory or parallel.
|
||||
*/
|
||||
if (result.isConvertibleToTwoLevel() && worth_convert_to_two_level)
|
||||
result.convertToTwoLevel();
|
||||
|
||||
/// Проверка ограничений.
|
||||
/// Checking the constraints.
|
||||
if (!checkLimits(result_size, no_more_keys))
|
||||
return false;
|
||||
|
||||
/** Сброс данных на диск, если потребляется слишком много оперативки.
|
||||
* Данные можно сбросить на диск только если используется двухуровневая структура агрегации.
|
||||
/** Flush data to disk if too much RAM is consumed.
|
||||
* Data can only be flushed to disk if a two-level aggregation structure is used.
|
||||
*/
|
||||
if (params.max_bytes_before_external_group_by
|
||||
&& result.isTwoLevel()
|
||||
@ -889,7 +889,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, si
|
||||
LOG_DEBUG(log, "Writing part of aggregation data into temporary file " << path << ".");
|
||||
ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart);
|
||||
|
||||
/// Сбрасываем только двухуровневые данные.
|
||||
/// Flush only two-level data.
|
||||
|
||||
#define M(NAME) \
|
||||
else if (data_variants.type == AggregatedDataVariants::Type::NAME) \
|
||||
@ -901,7 +901,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, si
|
||||
else
|
||||
throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT);
|
||||
|
||||
/// NOTE Вместо освобождения памяти и создания новых хэш-таблиц и арены, можно переиспользовать старые.
|
||||
/// NOTE Instead of freeing up memory and creating new hash tables and arenas, you can re-use the old ones.
|
||||
data_variants.init(data_variants.type);
|
||||
data_variants.aggregates_pools = Arenas(1, std::make_shared<Arena>());
|
||||
data_variants.aggregates_pool = data_variants.aggregates_pools.back().get();
|
||||
@ -986,7 +986,7 @@ void Aggregator::writeToTemporaryFileImpl(
|
||||
max_temporary_block_size_bytes = block_size_bytes;
|
||||
}
|
||||
|
||||
/// data_variants не будет уничтожать состояния агрегатных функций в деструкторе. Теперь состояниями владеют ColumnAggregateFunction.
|
||||
/// `data_variants` will not destroy the states of aggregate functions in the destructor. Now the states own the ColumnAggregateFunction.
|
||||
data_variants.aggregator = nullptr;
|
||||
|
||||
LOG_TRACE(log, std::fixed << std::setprecision(3)
|
||||
@ -1025,10 +1025,10 @@ void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & re
|
||||
AggregateColumns aggregate_columns(params.aggregates_size);
|
||||
Sizes key_sizes;
|
||||
|
||||
/** Используется, если есть ограничение на максимальное количество строк при агрегации,
|
||||
* и если group_by_overflow_mode == ANY.
|
||||
* В этом случае, новые ключи не добавляются в набор, а производится агрегация только по
|
||||
* ключам, которые уже успели попасть в набор.
|
||||
/** Used if there is a limit on the maximum number of rows in the aggregation,
|
||||
* and if group_by_overflow_mode == ANY.
|
||||
* In this case, new keys are not added to the set, but aggregation is performed only by
|
||||
* keys that have already managed to get into the set.
|
||||
*/
|
||||
bool no_more_keys = false;
|
||||
|
||||
@ -1039,7 +1039,7 @@ void Aggregator::execute(BlockInputStreamPtr stream, AggregatedDataVariants & re
|
||||
size_t src_rows = 0;
|
||||
size_t src_bytes = 0;
|
||||
|
||||
/// Читаем все данные
|
||||
/// Read all the data
|
||||
while (Block block = stream->read())
|
||||
{
|
||||
if (isCancelled())
|
||||
@ -1081,7 +1081,7 @@ void Aggregator::convertToBlockImpl(
|
||||
else
|
||||
convertToBlockImplNotFinal(method, data, key_columns, aggregate_columns, key_sizes);
|
||||
|
||||
/// Для того, чтобы пораньше освободить память.
|
||||
/// In order to release memory early.
|
||||
data.clearAndShrink();
|
||||
}
|
||||
|
||||
@ -1104,7 +1104,7 @@ void NO_INLINE Aggregator::convertToBlockImplFinal(
|
||||
*final_aggregate_columns[i]);
|
||||
}
|
||||
|
||||
destroyImpl(method, data); /// NOTE Можно сделать лучше.
|
||||
destroyImpl(method, data); /// NOTE You can do better.
|
||||
}
|
||||
|
||||
template <typename Method, typename Table>
|
||||
@ -1120,7 +1120,7 @@ void NO_INLINE Aggregator::convertToBlockImplNotFinal(
|
||||
{
|
||||
method.insertKeyIntoColumns(value, key_columns, params.keys_size, key_sizes);
|
||||
|
||||
/// reserved, поэтому push_back не кидает исключений
|
||||
/// reserved, so push_back does not throw exceptions
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i]->push_back(Method::getAggregateData(value.second) + offsets_of_aggregate_states[i]);
|
||||
|
||||
@ -1152,7 +1152,7 @@ Block Aggregator::prepareBlockAndFill(
|
||||
{
|
||||
if (!final)
|
||||
{
|
||||
/// Столбец ColumnAggregateFunction захватывает разделяемое владение ареной с состояниями агрегатных функций.
|
||||
/// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states.
|
||||
ColumnAggregateFunction & column_aggregate_func = static_cast<ColumnAggregateFunction &>(
|
||||
*res.safeGetByPosition(i + params.keys_size).column);
|
||||
|
||||
@ -1171,7 +1171,7 @@ Block Aggregator::prepareBlockAndFill(
|
||||
|
||||
if (aggregate_functions[i]->isState())
|
||||
{
|
||||
/// Столбец ColumnAggregateFunction захватывает разделяемое владение ареной с состояниями агрегатных функций.
|
||||
/// The ColumnAggregateFunction column captures the shared ownership of the arena with aggregate function states.
|
||||
ColumnAggregateFunction & column_aggregate_func = static_cast<ColumnAggregateFunction &>(*column.column);
|
||||
|
||||
for (size_t j = 0; j < data_variants.aggregates_pools.size(); ++j)
|
||||
@ -1184,7 +1184,7 @@ Block Aggregator::prepareBlockAndFill(
|
||||
|
||||
filler(key_columns, aggregate_columns, final_aggregate_columns, data_variants.key_sizes, final);
|
||||
|
||||
/// Изменяем размер столбцов-констант в блоке.
|
||||
/// Change the size of the columns-constants in the block.
|
||||
size_t columns = res.columns();
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
if (res.safeGetByPosition(i).column->isConst())
|
||||
@ -1295,7 +1295,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
|
||||
return convertOneBucketToBlock(data_variants, method, final, bucket);
|
||||
};
|
||||
|
||||
/// packaged_task используются, чтобы исключения автоматически прокидывались в основной поток.
|
||||
/// packaged_task is used to ensure that exceptions are automatically thrown into the main stream.
|
||||
|
||||
std::vector<std::packaged_task<Block()>> tasks(Method::Data::NUM_BUCKETS);
|
||||
|
||||
@ -1316,7 +1316,7 @@ BlocksList Aggregator::prepareBlocksAndFillTwoLevelImpl(
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
/// Если этого не делать, то в случае исключения, tasks уничтожится раньше завершения потоков, и будет плохо.
|
||||
/// If this is not done, then in case of an exception, tasks will be destroyed before the threads are completed, and it will be bad.
|
||||
if (thread_pool)
|
||||
thread_pool->wait();
|
||||
|
||||
@ -1351,13 +1351,13 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
|
||||
|
||||
BlocksList blocks;
|
||||
|
||||
/// В какой структуре данных агрегированы данные?
|
||||
/// In what data structure is the data aggregated?
|
||||
if (data_variants.empty())
|
||||
return blocks;
|
||||
|
||||
std::unique_ptr<ThreadPool> thread_pool;
|
||||
if (max_threads > 1 && data_variants.sizeWithoutOverflowRow() > 100000 /// TODO Сделать настраиваемый порог.
|
||||
&& data_variants.isTwoLevel()) /// TODO Использовать общий тред-пул с функцией merge.
|
||||
if (max_threads > 1 && data_variants.sizeWithoutOverflowRow() > 100000 /// TODO Make a custom threshold.
|
||||
&& data_variants.isTwoLevel()) /// TODO Use the shared thread pool with the `merge` function.
|
||||
thread_pool = std::make_unique<ThreadPool>(max_threads);
|
||||
|
||||
if (isCancelled())
|
||||
@ -1380,8 +1380,8 @@ BlocksList Aggregator::convertToBlocks(AggregatedDataVariants & data_variants, b
|
||||
|
||||
if (!final)
|
||||
{
|
||||
/// data_variants не будет уничтожать состояния агрегатных функций в деструкторе.
|
||||
/// Теперь состояниями владеют ColumnAggregateFunction.
|
||||
/// data_variants will not destroy the states of aggregate functions in the destructor.
|
||||
/// Now ColumnAggregateFunction owns the states.
|
||||
data_variants.aggregator = nullptr;
|
||||
}
|
||||
|
||||
@ -1512,7 +1512,7 @@ void NO_INLINE Aggregator::mergeWithoutKeyDataImpl(
|
||||
{
|
||||
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
||||
|
||||
/// Все результаты агрегации соединяем с первым.
|
||||
/// We connect all aggregation results to the first.
|
||||
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
||||
{
|
||||
AggregatedDataWithoutKey & res_data = res->without_key;
|
||||
@ -1536,7 +1536,7 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
|
||||
AggregatedDataVariantsPtr & res = non_empty_data[0];
|
||||
bool no_more_keys = false;
|
||||
|
||||
/// Все результаты агрегации соединяем с первым.
|
||||
/// We connect all aggregation results to the first.
|
||||
for (size_t i = 1, size = non_empty_data.size(); i < size; ++i)
|
||||
{
|
||||
if (!checkLimits(res->sizeWithoutOverflowRow(), no_more_keys))
|
||||
@ -1561,7 +1561,7 @@ void NO_INLINE Aggregator::mergeSingleLevelDataImpl(
|
||||
getDataVariant<Method>(current).data,
|
||||
res->aggregates_pool);
|
||||
|
||||
/// current не будет уничтожать состояния агрегатных функций в деструкторе
|
||||
/// `current` will not destroy the states of aggregate functions in the destructor
|
||||
current.aggregator = nullptr;
|
||||
}
|
||||
}
|
||||
@ -1571,7 +1571,7 @@ template <typename Method>
|
||||
void NO_INLINE Aggregator::mergeBucketImpl(
|
||||
ManyAggregatedDataVariants & data, Int32 bucket, Arena * arena) const
|
||||
{
|
||||
/// Все результаты агрегации соединяем с первым.
|
||||
/// We connect all aggregation results to the first.
|
||||
AggregatedDataVariantsPtr & res = data[0];
|
||||
for (size_t i = 1, size = data.size(); i < size; ++i)
|
||||
{
|
||||
@ -1585,16 +1585,16 @@ void NO_INLINE Aggregator::mergeBucketImpl(
|
||||
}
|
||||
|
||||
|
||||
/** Объединят вместе состояния агрегации, превращает их в блоки и выдаёт потоково.
|
||||
* Если состояния агрегации двухуровневые, то выдаёт блоки строго по порядку bucket_num.
|
||||
* (Это важно при распределённой обработке.)
|
||||
* При этом, может обрабатывать разные bucket-ы параллельно, используя до threads потоков.
|
||||
/** Combines aggregation states together, turns them into blocks, and outputs streams.
|
||||
* If the aggregation states are two-level, then it produces blocks strictly in order bucket_num.
|
||||
* (This is important for distributed processing.)
|
||||
* In doing so, it can handle different buckets in parallel, using up to `threads` threads.
|
||||
*/
|
||||
class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
/** На вход подаётся набор непустых множеств частично агрегированных данных,
|
||||
* которые все либо являются одноуровневыми, либо являются двухуровневыми.
|
||||
/** The input is a set of non-empty sets of partially aggregated data,
|
||||
* which are all either single-level, or are two-level.
|
||||
*/
|
||||
MergingAndConvertingBlockInputStream(const Aggregator & aggregator_, ManyAggregatedDataVariants & data_, bool final_, size_t threads_)
|
||||
: aggregator(aggregator_), data(data_), final(final_), threads(threads_)
|
||||
@ -1803,7 +1803,7 @@ std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
|
||||
|
||||
if (non_empty_data.size() > 1)
|
||||
{
|
||||
/// Отсортируем состояния по убыванию размера, чтобы мердж был более эффективным (так как все состояния мерджатся в первое).
|
||||
/// Sort the states in descending order so that the merge is more efficient (since all states are merged into the first).
|
||||
std::sort(non_empty_data.begin(), non_empty_data.end(),
|
||||
[](const AggregatedDataVariantsPtr & lhs, const AggregatedDataVariantsPtr & rhs)
|
||||
{
|
||||
@ -1811,8 +1811,8 @@ std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
|
||||
});
|
||||
}
|
||||
|
||||
/// Если хотя бы один из вариантов двухуровневый, то переконвертируем все варианты в двухуровневые, если есть не такие.
|
||||
/// Замечание - возможно, было бы более оптимально не конвертировать одноуровневые варианты перед мерджем, а мерджить их отдельно, в конце.
|
||||
/// If at least one of the options is two-level, then convert all the options into two-level ones, if there are not such.
|
||||
/// Note - perhaps it would be more optimal not to convert single-level versions before the merge, but merge them separately, at the end.
|
||||
|
||||
bool has_at_least_one_two_level = false;
|
||||
for (const auto & variant : non_empty_data)
|
||||
@ -1836,8 +1836,8 @@ std::unique_ptr<IBlockInputStream> Aggregator::mergeAndConvertToBlocks(
|
||||
if (first->type != non_empty_data[i]->type)
|
||||
throw Exception("Cannot merge different aggregated data variants.", ErrorCodes::CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS);
|
||||
|
||||
/** В первое множество данных могут быть перемещены элементы из остальных множеств.
|
||||
* Поэтому, оно должно владеть всеми аренами всех остальных множеств.
|
||||
/** Elements from the remaining sets can be moved to the first data set.
|
||||
* Therefore, it must own all the arenas of all other sets.
|
||||
*/
|
||||
first->aggregates_pools.insert(first->aggregates_pools.end(),
|
||||
non_empty_data[i]->aggregates_pools.begin(), non_empty_data[i]->aggregates_pools.end());
|
||||
@ -1859,7 +1859,7 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
|
||||
ConstColumnPlainPtrs key_columns(params.keys_size);
|
||||
AggregateColumnsData aggregate_columns(params.aggregates_size);
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
key_columns[i] = block.safeGetByPosition(i).column.get();
|
||||
|
||||
@ -1869,17 +1869,17 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
|
||||
typename Method::State state;
|
||||
state.init(key_columns);
|
||||
|
||||
/// Для всех строчек.
|
||||
/// For all rows.
|
||||
StringRefs keys(params.keys_size);
|
||||
size_t rows = block.rows();
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
typename Table::iterator it;
|
||||
|
||||
bool inserted; /// Вставили новый ключ, или такой ключ уже был?
|
||||
bool overflow = false; /// Новый ключ не поместился в хэш-таблицу из-за no_more_keys.
|
||||
bool inserted; /// Inserted a new key, or was this key already?
|
||||
bool overflow = false; /// The new key did not fit in the hash table because of no_more_keys.
|
||||
|
||||
/// Получаем ключ для вставки в хэш-таблицу.
|
||||
/// Get the key to insert into the hash table.
|
||||
auto key = state.getKey(key_columns, params.keys_size, i, key_sizes, keys, *aggregates_pool);
|
||||
|
||||
if (!no_more_keys)
|
||||
@ -1894,14 +1894,14 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
|
||||
overflow = true;
|
||||
}
|
||||
|
||||
/// Если ключ не поместился, и данные не надо агрегировать в отдельную строку, то делать нечего.
|
||||
/// If the key does not fit, and the data does not need to be aggregated into a separate row, then there's nothing to do.
|
||||
if (no_more_keys && overflow && !overflow_row)
|
||||
{
|
||||
method.onExistingKey(key, keys, *aggregates_pool);
|
||||
continue;
|
||||
}
|
||||
|
||||
/// Если вставили новый ключ - инициализируем состояния агрегатных функций, и возможно, что-нибудь связанное с ключом.
|
||||
/// If a new key is inserted, initialize the states of the aggregate functions, and possibly something related to the key.
|
||||
if (inserted)
|
||||
{
|
||||
AggregateDataPtr & aggregate_data = Method::getAggregateData(it->second);
|
||||
@ -1918,7 +1918,7 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
|
||||
|
||||
AggregateDataPtr value = (!no_more_keys || !overflow) ? Method::getAggregateData(it->second) : overflow_row;
|
||||
|
||||
/// Мерджим состояния агрегатных функций.
|
||||
/// Merge state of aggregate functions.
|
||||
for (size_t j = 0; j < params.aggregates_size; ++j)
|
||||
aggregate_functions[j]->merge(
|
||||
value + offsets_of_aggregate_states[j],
|
||||
@ -1926,7 +1926,7 @@ void NO_INLINE Aggregator::mergeStreamsImplCase(
|
||||
aggregates_pool);
|
||||
}
|
||||
|
||||
/// Пораньше освобождаем память.
|
||||
/// Early release memory.
|
||||
block.clear();
|
||||
}
|
||||
|
||||
@ -1953,7 +1953,7 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
|
||||
{
|
||||
AggregateColumnsData aggregate_columns(params.aggregates_size);
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_columns[i] = &typeid_cast<ColumnAggregateFunction &>(*block.safeGetByPosition(params.keys_size + i).column).getData();
|
||||
|
||||
@ -1965,11 +1965,11 @@ void NO_INLINE Aggregator::mergeWithoutKeyStreamsImpl(
|
||||
res = place;
|
||||
}
|
||||
|
||||
/// Добавляем значения
|
||||
/// Adding Values
|
||||
for (size_t i = 0; i < params.aggregates_size; ++i)
|
||||
aggregate_functions[i]->merge(res + offsets_of_aggregate_states[i], (*aggregate_columns[i])[0], result.aggregates_pool);
|
||||
|
||||
/// Пораньше освобождаем память.
|
||||
/// Early release memory.
|
||||
block.clear();
|
||||
}
|
||||
|
||||
@ -1989,15 +1989,15 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
if (isCancelled())
|
||||
return;
|
||||
|
||||
/** Если на удалённых серверах использовался двухуровневый метод агрегации,
|
||||
* то в блоках будет расположена информация о номере корзины.
|
||||
* Тогда вычисления можно будет распараллелить по корзинам.
|
||||
* Разложим блоки по указанным в них номерам корзин.
|
||||
/** If the remote servers used a two-level aggregation method,
|
||||
* then blocks will contain information about the number of the bucket.
|
||||
* Then the calculations can be parallelized by buckets.
|
||||
* We decompose the blocks to the bucket numbers indicated in them.
|
||||
*/
|
||||
using BucketToBlocks = std::map<Int32, BlocksList>;
|
||||
BucketToBlocks bucket_to_blocks;
|
||||
|
||||
/// Читаем все данные.
|
||||
/// Read all the data.
|
||||
LOG_TRACE(log, "Reading blocks of partially aggregated data.");
|
||||
|
||||
size_t total_input_rows = 0;
|
||||
@ -2019,16 +2019,16 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
|
||||
setSampleBlock(bucket_to_blocks.begin()->second.front());
|
||||
|
||||
/// Каким способом выполнять агрегацию?
|
||||
/// How to perform the aggregation?
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
key_columns[i] = sample.safeGetByPosition(i).column.get();
|
||||
|
||||
Sizes key_sizes;
|
||||
AggregatedDataVariants::Type method = chooseAggregationMethod(key_columns, key_sizes);
|
||||
|
||||
/** Минус единицей обозначается отсутствие информации о корзине
|
||||
* - в случае одноуровневой агрегации, а также для блоков с "переполнившимися" значениями.
|
||||
* Если есть хотя бы один блок с номером корзины больше нуля, значит была двухуровневая агрегация.
|
||||
/** `minus one` means the absence of information about the bucket
|
||||
* - in the case of single-level aggregation, as well as for blocks with "overflowing" values.
|
||||
* If there is at least one block with a bucket number greater than zero, then there was a two-level aggregation.
|
||||
*/
|
||||
auto max_bucket = bucket_to_blocks.rbegin()->first;
|
||||
size_t has_two_level = max_bucket > 0;
|
||||
@ -2047,7 +2047,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
if (isCancelled())
|
||||
return;
|
||||
|
||||
/// result будет уничтожать состояния агрегатных функций в деструкторе
|
||||
/// result will destroy the states of aggregate functions in the destructor
|
||||
result.aggregator = this;
|
||||
|
||||
result.init(method);
|
||||
@ -2056,12 +2056,12 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
|
||||
bool has_blocks_with_unknown_bucket = bucket_to_blocks.count(-1);
|
||||
|
||||
/// Сначала параллельно мерджим для отдельных bucket-ов. Затем домердживаем данные, не распределённые по bucket-ам.
|
||||
/// First, parallel the merge for the individual buckets. Then we continue merge the data not allocated to the buckets.
|
||||
if (has_two_level)
|
||||
{
|
||||
/** В этом случае, no_more_keys не поддерживается в связи с тем, что
|
||||
* из разных потоков трудно обновлять общее состояние для "остальных" ключей (overflows).
|
||||
* То есть, ключей в итоге может оказаться существенно больше, чем max_rows_to_group_by.
|
||||
/** In this case, no_more_keys is not supported due to the fact that
|
||||
* from different threads it is difficult to update the general state for "other" keys (overflows).
|
||||
* That is, the keys in the end can be significantly larger than max_rows_to_group_by.
|
||||
*/
|
||||
|
||||
LOG_TRACE(log, "Merging partially aggregated two-level data.");
|
||||
@ -2088,7 +2088,7 @@ void Aggregator::mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants
|
||||
};
|
||||
|
||||
std::unique_ptr<ThreadPool> thread_pool;
|
||||
if (max_threads > 1 && total_input_rows > 100000 /// TODO Сделать настраиваемый порог.
|
||||
if (max_threads > 1 && total_input_rows > 100000 /// TODO Make a custom threshold.
|
||||
&& has_two_level)
|
||||
thread_pool = std::make_unique<ThreadPool>(max_threads);
|
||||
|
||||
@ -2171,7 +2171,7 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
initialize({});
|
||||
setSampleBlock(blocks.front());
|
||||
|
||||
/// Каким способом выполнять агрегацию?
|
||||
/// How to perform the aggregation?
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
key_columns[i] = sample.safeGetByPosition(i).column.get();
|
||||
|
||||
@ -2201,10 +2201,10 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
|
||||
#undef APPLY_FOR_VARIANTS_THAT_MAY_USE_BETTER_HASH_FUNCTION
|
||||
|
||||
/// Временные данные для агрегации.
|
||||
/// Temporary data for aggregation.
|
||||
AggregatedDataVariants result;
|
||||
|
||||
/// result будет уничтожать состояния агрегатных функций в деструкторе
|
||||
/// result will destroy the states of aggregate functions in the destructor
|
||||
result.aggregator = this;
|
||||
|
||||
result.init(method);
|
||||
@ -2233,12 +2233,12 @@ Block Aggregator::mergeBlocks(BlocksList & blocks, bool final)
|
||||
|
||||
if (merged_blocks.size() > 1)
|
||||
{
|
||||
/** Может быть два блока. Один с is_overflows, другой - нет.
|
||||
* Если есть непустой блок не is_overflows, то удаляем блок с is_overflows.
|
||||
* Если есть пустой блок не is_overflows и блок с is_overflows, то удаляем пустой блок.
|
||||
/** There may be two blocks. One is with `is_overflows`, the other is not.
|
||||
* If there is a non-empty block not is_overflows, then delete the block with is_overflows.
|
||||
* If there is an empty block not is_overflows and a block with is_overflows, then delete the empty block.
|
||||
*
|
||||
* Это делаем, потому что исходим из допущения, что в функцию передаются
|
||||
* либо все блоки не is_overflows, либо все блоки is_overflows.
|
||||
* Do this, because we start from the assumption that the function is passed to the function
|
||||
* either all the blocks are not is_overflows, or all the blocks is_overflows.
|
||||
*/
|
||||
|
||||
bool has_nonempty_nonoverflows = false;
|
||||
@ -2362,7 +2362,7 @@ std::vector<Block> Aggregator::convertBlockToTwoLevel(const Block & block)
|
||||
ConstColumnPlainPtrs key_columns(params.keys_size);
|
||||
Sizes key_sizes;
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < params.keys_size; ++i)
|
||||
key_columns[i] = block.safeGetByPosition(i).column.get();
|
||||
|
||||
@ -2420,9 +2420,9 @@ void NO_INLINE Aggregator::destroyImpl(
|
||||
{
|
||||
AggregateDataPtr & data = Method::getAggregateData(elem.second);
|
||||
|
||||
/** Если исключение (обычно нехватка памяти, кидается MemoryTracker-ом) возникло
|
||||
* после вставки ключа в хэш-таблицу, но до создания всех состояний агрегатных функций,
|
||||
* то data будет равен nullptr-у.
|
||||
/** If an exception (usually a lack of memory, the MemoryTracker throws) arose
|
||||
* after inserting the key into a hash table, but before creating all states of aggregate functions,
|
||||
* then data will be equal nullptr.
|
||||
*/
|
||||
if (nullptr == data)
|
||||
continue;
|
||||
@ -2458,7 +2458,7 @@ void Aggregator::destroyAllAggregateStates(AggregatedDataVariants & result)
|
||||
|
||||
LOG_TRACE(log, "Destroying aggregate states");
|
||||
|
||||
/// В какой структуре данных агрегированы данные?
|
||||
/// In what data structure is the data aggregated?
|
||||
if (result.type == AggregatedDataVariants::Type::without_key || params.overflow_row)
|
||||
destroyWithoutKey(result);
|
||||
|
||||
|
@ -158,7 +158,7 @@ Clusters::Impl Clusters::getContainer() const
|
||||
return impl;
|
||||
}
|
||||
|
||||
/// Реализация класса Cluster
|
||||
/// Implementation of `Cluster` class
|
||||
|
||||
Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & settings, const String & cluster_name)
|
||||
{
|
||||
@ -176,7 +176,7 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
|
||||
{
|
||||
if (startsWith(key, "node"))
|
||||
{
|
||||
/// Шард без реплик.
|
||||
/// Shard without replicas.
|
||||
|
||||
const auto & prefix = config_prefix + key;
|
||||
const auto weight = config.getInt(prefix + ".weight", default_weight);
|
||||
@ -211,7 +211,7 @@ Cluster::Cluster(Poco::Util::AbstractConfiguration & config, const Settings & se
|
||||
}
|
||||
else if (startsWith(key, "shard"))
|
||||
{
|
||||
/// Шард с репликами.
|
||||
/// Shard with replicas.
|
||||
|
||||
Poco::Util::AbstractConfiguration::Keys replica_keys;
|
||||
config.keys(config_prefix + key, replica_keys);
|
||||
|
@ -33,10 +33,10 @@ BlockInputStreamPtr DescribeQueryConstructor::createLocal(ASTPtr query_ast, cons
|
||||
InterpreterDescribeQuery interpreter{query_ast, context};
|
||||
BlockInputStreamPtr stream = interpreter.execute().in;
|
||||
|
||||
/** Материализация нужна, так как с удалённых серверов константы приходят материализованными.
|
||||
* Если этого не делать, то в разных потоках будут получаться разные типы (Const и не-Const) столбцов,
|
||||
* а это не разрешено, так как весь код исходит из допущения, что в потоке блоков все типы одинаковые.
|
||||
*/
|
||||
/** Materialization is needed, since from remote servers the constants come materialized.
|
||||
* If you do not do this, different types (Const and non-Const) columns will be produced in different threads,
|
||||
* And this is not allowed, since all code is based on the assumption that in the block stream all types are the same.
|
||||
*/
|
||||
BlockInputStreamPtr materialized_stream = std::make_shared<MaterializingBlockInputStream>(stream);
|
||||
|
||||
return std::make_shared<BlockExtraInfoInputStream>(materialized_stream, toBlockExtraInfo(address));
|
||||
|
@ -27,10 +27,10 @@ BlockInputStreamPtr SelectQueryConstructor::createLocal(ASTPtr query_ast, const
|
||||
InterpreterSelectQuery interpreter{query_ast, context, processed_stage};
|
||||
BlockInputStreamPtr stream = interpreter.execute().in;
|
||||
|
||||
/** Материализация нужна, так как с удалённых серверов константы приходят материализованными.
|
||||
* Если этого не делать, то в разных потоках будут получаться разные типы (Const и не-Const) столбцов,
|
||||
* а это не разрешено, так как весь код исходит из допущения, что в потоке блоков все типы одинаковые.
|
||||
*/
|
||||
/** Materialization is needed, since from remote servers the constants come materialized.
|
||||
* If you do not do this, different types (Const and non-Const) columns will be produced in different threads,
|
||||
* And this is not allowed, since all code is based on the assumption that in the block stream all types are the same.
|
||||
*/
|
||||
return std::make_shared<MaterializingBlockInputStream>(stream);
|
||||
}
|
||||
|
||||
|
@ -64,7 +64,7 @@ static Compiler::HashedKey getHash(const std::string & key)
|
||||
}
|
||||
|
||||
|
||||
/// Без расширения .so.
|
||||
/// Without .so extension.
|
||||
static std::string hashedKeyToFileName(Compiler::HashedKey hashed_key)
|
||||
{
|
||||
std::string file_name;
|
||||
@ -91,19 +91,19 @@ SharedLibraryPtr Compiler::getOrCount(
|
||||
|
||||
UInt32 count = ++counts[hashed_key];
|
||||
|
||||
/// Есть готовая открытая библиотека? Или, если библиотека в процессе компиляции, там будет nullptr.
|
||||
/// Is there a ready open library? Or, if the library is in the process of compiling, there will be nullptr.
|
||||
Libraries::iterator it = libraries.find(hashed_key);
|
||||
if (libraries.end() != it)
|
||||
{
|
||||
if (!it->second)
|
||||
LOG_INFO(log, "Library " << hashedKeyToFileName(hashed_key) << " is already compiling or compilation was failed.");
|
||||
|
||||
/// TODO В этом случае, после окончания компиляции, не будет дёрнут колбэк.
|
||||
/// TODO In this case, after the compilation is finished, the callback will not be called.
|
||||
|
||||
return it->second;
|
||||
}
|
||||
|
||||
/// Есть файл с библиотекой, оставшийся от предыдущего запуска?
|
||||
/// Is there a file with the library left over from the previous launch?
|
||||
std::string file_name = hashedKeyToFileName(hashed_key);
|
||||
if (files.count(file_name))
|
||||
{
|
||||
@ -115,15 +115,15 @@ SharedLibraryPtr Compiler::getOrCount(
|
||||
return lib;
|
||||
}
|
||||
|
||||
/// Достигнуто ли min_count_to_compile?
|
||||
/// Has min_count_to_compile been reached?
|
||||
if (count >= min_count_to_compile)
|
||||
{
|
||||
/// Значение min_count_to_compile, равное нулю, обозначает необходимость синхронной компиляции.
|
||||
/// The min_count_to_compile value of zero indicates the need for synchronous compilation.
|
||||
|
||||
/// Есть ли свободные потоки.
|
||||
/// Are there any free threads?
|
||||
if (min_count_to_compile == 0 || pool.active() < pool.size())
|
||||
{
|
||||
/// Обозначает, что библиотека в процессе компиляции.
|
||||
/// Indicates that the library is in the process of compiling.
|
||||
libraries[hashed_key] = nullptr;
|
||||
|
||||
LOG_INFO(log, "Compiling code " << file_name << ", key: " << key);
|
||||
@ -181,7 +181,7 @@ void Compiler::compile(
|
||||
|
||||
std::stringstream command;
|
||||
|
||||
/// Слегка неудобно.
|
||||
/// Slightly uncomfortable.
|
||||
command <<
|
||||
"LD_LIBRARY_PATH=" PATH_SHARE "/clickhouse/bin/"
|
||||
" " INTERNAL_COMPILER_EXECUTABLE
|
||||
@ -219,7 +219,7 @@ void Compiler::compile(
|
||||
if (!compile_result.empty())
|
||||
throw Exception("Cannot compile code:\n\n" + command.str() + "\n\n" + compile_result);
|
||||
|
||||
/// Если до этого была ошибка, то файл с кодом остаётся для возможности просмотра.
|
||||
/// If there was an error before, the file with the code remains for viewing.
|
||||
Poco::File(cpp_file_path).remove();
|
||||
|
||||
Poco::File(so_tmp_file_path).renameTo(so_file_path);
|
||||
|
@ -94,37 +94,37 @@ struct ContextShared
|
||||
/// Separate mutex for re-initialization of zookeer session. This operation could take a long time and must not interfere with another operations.
|
||||
mutable std::mutex zookeeper_mutex;
|
||||
|
||||
mutable zkutil::ZooKeeperPtr zookeeper; /// Клиент для ZooKeeper.
|
||||
mutable zkutil::ZooKeeperPtr zookeeper; /// Client for ZooKeeper.
|
||||
|
||||
String interserver_io_host; /// Имя хоста по которым это сервер доступен для других серверов.
|
||||
int interserver_io_port; /// и порт,
|
||||
String interserver_io_host; /// The host name by which this server is available for other servers.
|
||||
int interserver_io_port; /// and port,
|
||||
|
||||
String path; /// Путь к директории с данными, со слешем на конце.
|
||||
String tmp_path; /// Путь ко временным файлам, возникающим при обработке запроса.
|
||||
String path; /// Path to the data directory, with a slash at the end.
|
||||
String tmp_path; /// The path to the temporary files that occur when processing the request.
|
||||
String flags_path; ///
|
||||
Databases databases; /// Список БД и таблиц в них.
|
||||
TableFunctionFactory table_function_factory; /// Табличные функции.
|
||||
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
|
||||
FormatFactory format_factory; /// Форматы.
|
||||
Databases databases; /// List of databases and tables in them.
|
||||
TableFunctionFactory table_function_factory; /// Table functions.
|
||||
AggregateFunctionFactory aggregate_function_factory; /// Aggregate functions.
|
||||
FormatFactory format_factory; /// Formats.
|
||||
mutable std::shared_ptr<EmbeddedDictionaries> embedded_dictionaries; /// Metrica's dictionaeis. Have lazy initialization.
|
||||
mutable std::shared_ptr<ExternalDictionaries> external_dictionaries;
|
||||
String default_profile_name; /// Default profile name used for default values.
|
||||
Users users; /// Known users.
|
||||
Quotas quotas; /// Известные квоты на использование ресурсов.
|
||||
mutable UncompressedCachePtr uncompressed_cache; /// Кэш разжатых блоков.
|
||||
mutable MarkCachePtr mark_cache; /// Кэш засечек в сжатых файлах.
|
||||
ProcessList process_list; /// Исполняющиеся в данный момент запросы.
|
||||
MergeList merge_list; /// Список выполняемых мерджей (для (Replicated)?MergeTree)
|
||||
ViewDependencies view_dependencies; /// Текущие зависимости
|
||||
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
|
||||
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
|
||||
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
|
||||
Quotas quotas; /// Known quotas for resource use.
|
||||
mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks.
|
||||
mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files.
|
||||
ProcessList process_list; /// Executing queries at the moment.
|
||||
MergeList merge_list; /// The list of executable merge (for (Replicated)?MergeTree)
|
||||
ViewDependencies view_dependencies; /// Current dependencies
|
||||
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
|
||||
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
|
||||
BackgroundProcessingPoolPtr background_pool; /// The thread pool for the background work performed by the tables.
|
||||
ReshardingWorkerPtr resharding_worker;
|
||||
Macros macros; /// Substitutions extracted from config.
|
||||
std::unique_ptr<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary.
|
||||
std::unique_ptr<QueryLog> query_log; /// Used to log queries.
|
||||
std::shared_ptr<PartLog> part_log; /// Used to log operations with parts
|
||||
/// Правила для выбора метода сжатия в зависимости от размера куска.
|
||||
/// Rules for selecting the compression method, depending on the size of the part.
|
||||
mutable std::unique_ptr<CompressionMethodSelector> compression_method_selector;
|
||||
std::unique_ptr<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)
|
||||
@ -139,13 +139,13 @@ struct ContextShared
|
||||
|
||||
bool shutdown_called = false;
|
||||
|
||||
/// Позволяют запретить одновременное выполнение DDL запросов над одной и той же таблицей.
|
||||
/// Do not allow simultaneous execution of DDL requests on the same table.
|
||||
/// database -> table -> exception_message
|
||||
/// На время выполнения операции, сюда помещается элемент, и возвращается объект, который в деструкторе удаляет элемент.
|
||||
/// В случае, если элемент уже есть - кидается исключение. См. class DDLGuard ниже.
|
||||
/// For the duration of the operation, an element is placed here, and an object is returned, which deletes the element in the destructor.
|
||||
/// In case the element already exists, an exception is thrown. See class DDLGuard below.
|
||||
using DDLGuards = std::unordered_map<String, DDLGuard::Map>;
|
||||
DDLGuards ddl_guards;
|
||||
/// Если вы захватываете mutex и ddl_guards_mutex, то захватывать их нужно строго в этом порядке.
|
||||
/// If you capture mutex and ddl_guards_mutex, then you need to grab them strictly in this order.
|
||||
mutable std::mutex ddl_guards_mutex;
|
||||
|
||||
Stopwatch uptime_watch;
|
||||
@ -166,7 +166,7 @@ struct ContextShared
|
||||
}
|
||||
|
||||
|
||||
/** Выполнить сложную работу по уничтожению объектов заранее.
|
||||
/** Perform a complex job of destroying objects in advance.
|
||||
*/
|
||||
void shutdown()
|
||||
{
|
||||
@ -177,10 +177,10 @@ struct ContextShared
|
||||
query_log.reset();
|
||||
part_log.reset();
|
||||
|
||||
/** В этот момент, некоторые таблицы могут иметь потоки, которые блокируют наш mutex.
|
||||
* Чтобы корректно их завершить, скопируем текущий список таблиц,
|
||||
* и попросим их всех закончить свою работу.
|
||||
* Потом удалим все объекты с таблицами.
|
||||
/** At this point, some tables may have threads that block our mutex.
|
||||
* To complete them correctly, we will copy the current list of tables,
|
||||
* and ask them all to finish their work.
|
||||
* Then delete all objects with tables.
|
||||
*/
|
||||
|
||||
Databases current_databases;
|
||||
@ -382,8 +382,8 @@ void Context::checkDatabaseAccessRights(const std::string & database_name) const
|
||||
{
|
||||
if (client_info.current_user.empty() || (database_name == "system"))
|
||||
{
|
||||
/// Безымянный пользователь, т.е. сервер, имеет доступ ко всем БД.
|
||||
/// Все пользователи имеют доступ к БД system.
|
||||
/// An unnamed user, i.e. server, has access to all databases.
|
||||
/// All users have access to the database system.
|
||||
return;
|
||||
}
|
||||
if (!shared->users.isAllowedDatabase(client_info.current_user, database_name))
|
||||
@ -548,8 +548,8 @@ StoragePtr Context::getTableImpl(const String & database_name, const String & ta
|
||||
{
|
||||
auto lock = getLock();
|
||||
|
||||
/** Возможность обратиться к временным таблицам другого запроса в виде _query_QUERY_ID.table
|
||||
* NOTE В дальнейшем может потребоваться подумать об изоляции.
|
||||
/** Ability to access the temporary tables of another query in the form _query_QUERY_ID.table
|
||||
* NOTE In the future, you may need to think about isolation.
|
||||
*/
|
||||
if (startsWith(database_name, "_query_"))
|
||||
{
|
||||
@ -742,7 +742,7 @@ void Context::setCurrentQueryId(const String & query_id)
|
||||
throw Exception("Logical error: attempt to set query_id twice", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
String query_id_to_set = query_id;
|
||||
if (query_id_to_set.empty()) /// Если пользователь не передал свой query_id, то генерируем его самостоятельно.
|
||||
if (query_id_to_set.empty()) /// If the user did not submit his query_id, then we generate it ourselves.
|
||||
query_id_to_set = shared->uuid_generator.createRandom().toString();
|
||||
|
||||
auto lock = getLock();
|
||||
@ -770,7 +770,7 @@ const Macros& Context::getMacros() const
|
||||
|
||||
void Context::setMacros(Macros && macros)
|
||||
{
|
||||
/// Полагаемся, что это присваивание происходит один раз при старте сервера. Если это не так, нужно использовать мьютекс.
|
||||
/// We assume that this assignment occurs once when the server starts. If this is not the case, you need to use a mutex.
|
||||
shared->macros = macros;
|
||||
}
|
||||
|
||||
@ -855,7 +855,7 @@ void Context::tryCreateExternalDictionaries() const
|
||||
|
||||
void Context::setProgressCallback(ProgressCallback callback)
|
||||
{
|
||||
/// Колбек устанавливается на сессию или на запрос. В сессии одновременно обрабатывается только один запрос. Поэтому блокировка не нужна.
|
||||
/// Callback is set to a session or to a query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
|
||||
progress_callback = callback;
|
||||
}
|
||||
|
||||
@ -867,7 +867,7 @@ ProgressCallback Context::getProgressCallback() const
|
||||
|
||||
void Context::setProcessListElement(ProcessList::Element * elem)
|
||||
{
|
||||
/// Устанавливается на сессию или на запрос. В сессии одновременно обрабатывается только один запрос. Поэтому блокировка не нужна.
|
||||
/// Set to a session or query. In the session, only one query is processed at a time. Therefore, the lock is not needed.
|
||||
process_list_elem = elem;
|
||||
}
|
||||
|
||||
|
@ -46,11 +46,11 @@ bool EmbeddedDictionaries::reloadDictionary(MultiVersion<Dictionary> & dictionar
|
||||
|
||||
bool EmbeddedDictionaries::reloadImpl(const bool throw_on_error)
|
||||
{
|
||||
/** Если не удаётся обновить справочники, то несмотря на это, не кидаем исключение (используем старые справочники).
|
||||
* Если старых корректных справочников нет, то при использовании функций, которые от них зависят,
|
||||
* будет кидаться исключение.
|
||||
* Производится попытка загрузить каждый справочник по-отдельности.
|
||||
*/
|
||||
/** If you can not update the directories, then despite this, do not throw an exception (use the old directories).
|
||||
* If there are no old correct directories, then when using functions that depend on them,
|
||||
* will throw an exception.
|
||||
* An attempt is made to load each directory separately.
|
||||
*/
|
||||
|
||||
LOG_INFO(log, "Loading dictionaries.");
|
||||
|
||||
|
@ -173,11 +173,11 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
{
|
||||
// std::cerr << "preparing: " << toString() << std::endl;
|
||||
|
||||
/** Константные выражения следует вычислить, и положить результат в sample_block.
|
||||
* Для неконстантных столбцов, следует в качестве column в sample_block положить nullptr.
|
||||
/** Constant expressions should be evaluated, and put the result in sample_block.
|
||||
* For non-constant columns, put the nullptr as the column in sample_block.
|
||||
*
|
||||
* Тот факт, что только для константных выражений column != nullptr,
|
||||
* может использоваться в дальнейшем при оптимизации запроса.
|
||||
* The fact that only for constant expressions column != nullptr,
|
||||
* can be used later when optimizing the query.
|
||||
*/
|
||||
|
||||
switch (type)
|
||||
@ -221,7 +221,7 @@ void ExpressionAction::prepare(Block & sample_block)
|
||||
|
||||
function->execute(sample_block, arguments, prerequisites, result_position);
|
||||
|
||||
/// Если получилась не константа, на всякий случай будем считать результат неизвестным.
|
||||
/// If the result is not a constant, just in case, we will consider the result as unknown.
|
||||
ColumnWithTypeAndName & col = sample_block.safeGetByPosition(result_position);
|
||||
if (!col.column->isConst())
|
||||
col.column = nullptr;
|
||||
@ -357,7 +357,7 @@ void ExpressionAction::execute(Block & block) const
|
||||
if (!any_array)
|
||||
throw Exception("ARRAY JOIN of not array: " + *array_joined_columns.begin(), ErrorCodes::TYPE_MISMATCH);
|
||||
|
||||
/// Если LEFT ARRAY JOIN, то создаём столбцы, в которых пустые массивы заменены на массивы с одним элементом - значением по-умолчанию.
|
||||
/// If LEFT ARRAY JOIN, then we create columns in which empty arrays are replaced by arrays with one element - the default value.
|
||||
std::map<String, ColumnPtr> non_empty_array_columns;
|
||||
if (array_join_is_left)
|
||||
{
|
||||
@ -668,7 +668,7 @@ void ExpressionActions::execute(Block & block) const
|
||||
|
||||
void ExpressionActions::executeOnTotals(Block & block) const
|
||||
{
|
||||
/// Если в подзапросе для JOIN-а есть totals, а у нас нет, то возьмём блок со значениями по-умолчанию вместо totals.
|
||||
/// If there is `totals` in the subquery for JOIN, but we do not, then take the block with the default values instead of `totals`.
|
||||
if (!block)
|
||||
{
|
||||
bool has_totals_in_join = false;
|
||||
@ -691,7 +691,7 @@ void ExpressionActions::executeOnTotals(Block & block) const
|
||||
}
|
||||
}
|
||||
else
|
||||
return; /// Нечего JOIN-ить.
|
||||
return; /// There's nothing to JOIN.
|
||||
}
|
||||
|
||||
for (const auto & action : actions)
|
||||
@ -732,9 +732,9 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
final_columns.insert(name);
|
||||
}
|
||||
|
||||
/// Какие столбцы нужны, чтобы выполнить действия от текущего до последнего.
|
||||
/// Which columns are needed to perform actions from the current to the last.
|
||||
NameSet needed_columns = final_columns;
|
||||
/// Какие столбцы никто не будет трогать от текущего действия до последнего.
|
||||
/// Which columns nobody will touch from the current action to the last.
|
||||
NameSet unmodified_columns;
|
||||
|
||||
{
|
||||
@ -743,8 +743,8 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
unmodified_columns.insert(it->name);
|
||||
}
|
||||
|
||||
/// Будем идти с конца и поддерживать множество нужных на данном этапе столбцов.
|
||||
/// Будем выбрасывать ненужные действия, хотя обычно их нет по построению.
|
||||
/// Let's go from the end and maintain set of required columns at this stage.
|
||||
/// We will throw out unnecessary actions, although usually they are absent by construction.
|
||||
for (int i = static_cast<int>(actions.size()) - 1; i >= 0; --i)
|
||||
{
|
||||
ExpressionAction & action = actions[i];
|
||||
@ -757,9 +757,9 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
}
|
||||
else if (action.type == ExpressionAction::ARRAY_JOIN)
|
||||
{
|
||||
/// Не будем ARRAY JOIN-ить столбцы, которые дальше не используются.
|
||||
/// Обычно такие столбцы не используются и до ARRAY JOIN, и поэтому выбрасываются дальше в этой функции.
|
||||
/// Не будем убирать все столбцы, чтобы не потерять количество строк.
|
||||
/// Do not ARRAY JOIN columns that are not used anymore.
|
||||
/// Usually, such columns are not used until ARRAY JOIN, and therefore are ejected further in this function.
|
||||
/// We will not remove all the columns so as not to lose the number of rows.
|
||||
for (auto it = action.array_joined_columns.begin(); it != action.array_joined_columns.end();)
|
||||
{
|
||||
bool need = needed_columns.count(*it);
|
||||
@ -772,8 +772,8 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
needed_columns.insert(*it);
|
||||
unmodified_columns.erase(*it);
|
||||
|
||||
/// Если никакие результаты ARRAY JOIN не используются, принудительно оставим на выходе произвольный столбец,
|
||||
/// чтобы не потерять количество строк.
|
||||
/// If no ARRAY JOIN results are used, forcibly leave an arbitrary column at the output,
|
||||
/// so you do not lose the number of rows.
|
||||
if (!need)
|
||||
final_columns.insert(*it);
|
||||
|
||||
@ -786,7 +786,7 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
std::string out = action.result_name;
|
||||
if (!out.empty())
|
||||
{
|
||||
/// Если результат не используется и нет побочных эффектов, выбросим действие.
|
||||
/// If the result is not used and there are no side effects, throw out the action.
|
||||
if (!needed_columns.count(out) &&
|
||||
(action.type == ExpressionAction::APPLY_FUNCTION
|
||||
|| action.type == ExpressionAction::ADD_COLUMN
|
||||
@ -806,8 +806,8 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
unmodified_columns.erase(out);
|
||||
needed_columns.erase(out);
|
||||
|
||||
/** Если функция - константное выражение, то заменим действие на добавление столбца-константы - результата.
|
||||
* То есть, осуществляем constant folding.
|
||||
/** If the function is a constant expression, then replace the action by adding a column-constant - result.
|
||||
* That is, we perform constant folding.
|
||||
*/
|
||||
if (action.type == ExpressionAction::APPLY_FUNCTION && sample_block.has(out))
|
||||
{
|
||||
@ -828,11 +828,11 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
}
|
||||
}
|
||||
|
||||
/// Не будем выбрасывать все входные столбцы, чтобы не потерять количество строк в блоке.
|
||||
/// We will not throw out all the input columns, so as not to lose the number of rows in the block.
|
||||
if (needed_columns.empty() && !input_columns.empty())
|
||||
needed_columns.insert(getSmallestColumn(input_columns));
|
||||
|
||||
/// Не будем оставлять блок пустым, чтобы не потерять количество строк в нем.
|
||||
/// We will not leave the block empty so as not to lose the number of rows in it.
|
||||
if (final_columns.empty())
|
||||
final_columns.insert(getSmallestColumn(input_columns));
|
||||
|
||||
@ -853,9 +853,9 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
std::cerr << action.toString() << "\n";
|
||||
std::cerr << "\n";*/
|
||||
|
||||
/// Удаление ненужных временных столбцов.
|
||||
/// Deletes unnecessary temporary columns.
|
||||
|
||||
/// Если у столбца после выполнения функции refcount = 0, то его можно удалить.
|
||||
/// If the column after performing the function `refcount = 0`, it can be deleted.
|
||||
std::map<String, int> columns_refcount;
|
||||
|
||||
for (const auto & name : final_columns)
|
||||
@ -903,7 +903,7 @@ void ExpressionActions::finalize(const Names & output_columns)
|
||||
for (const auto & name : action.prerequisite_names)
|
||||
process(name);
|
||||
|
||||
/// Для projection тут нет уменьшения refcount, так как действие project заменяет имена у столбцов, по сути, уже удаляя их под старыми именами.
|
||||
/// For `projection`, there is no reduction in `refcount`, because the `project` action replaces the names of the columns, in effect, already deleting them under the old names.
|
||||
}
|
||||
|
||||
actions.swap(new_actions);
|
||||
@ -987,17 +987,17 @@ void ExpressionActions::optimizeArrayJoin()
|
||||
const size_t NONE = actions.size();
|
||||
size_t first_array_join = NONE;
|
||||
|
||||
/// Столбцы, для вычисления которых нужен arrayJoin.
|
||||
/// Действия для их добавления нельзя переместить левее arrayJoin.
|
||||
/// Columns that need to be evaluated for arrayJoin.
|
||||
/// Actions for adding them can not be moved to the left of the arrayJoin.
|
||||
NameSet array_joined_columns;
|
||||
|
||||
/// Столбцы, нужные для вычисления arrayJoin или тех, кто от него зависит.
|
||||
/// Действия для их удаления нельзя переместить левее arrayJoin.
|
||||
/// Columns needed to evaluate arrayJoin or those that depend on it.
|
||||
/// Actions to delete them can not be moved to the left of the arrayJoin.
|
||||
NameSet array_join_dependencies;
|
||||
|
||||
for (size_t i = 0; i < actions.size(); ++i)
|
||||
{
|
||||
/// Не будем перемещать действия правее проецирования (тем более, что их там обычно нет).
|
||||
/// Do not move the action to the right of the projection (the more that they are not usually there).
|
||||
if (actions[i].type == ExpressionAction::PROJECT)
|
||||
break;
|
||||
|
||||
@ -1043,19 +1043,19 @@ void ExpressionActions::optimizeArrayJoin()
|
||||
|
||||
if (actions[i].type == ExpressionAction::REMOVE_COLUMN)
|
||||
{
|
||||
/// Если удаляем столбец, не нужный для arrayJoin (и тех, кто от него зависит), можно его удалить до arrayJoin.
|
||||
/// If you delete a column that is not needed for arrayJoin (and those who depend on it), you can delete it before arrayJoin.
|
||||
can_move = !array_join_dependencies.count(actions[i].source_name);
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Если действие не удаляет столбцы и не зависит от результата arrayJoin, можно сделать его до arrayJoin.
|
||||
/// If the action does not delete the columns and does not depend on the result of arrayJoin, you can make it until arrayJoin.
|
||||
can_move = true;
|
||||
}
|
||||
|
||||
/// Переместим текущее действие в позицию сразу перед первым arrayJoin.
|
||||
/// Move the current action to the position just before the first arrayJoin.
|
||||
if (can_move)
|
||||
{
|
||||
/// Переместим i-й элемент в позицию first_array_join.
|
||||
/// Move the i-th element to the position `first_array_join`.
|
||||
std::rotate(actions.begin() + first_array_join, actions.begin() + i, actions.begin() + i + 1);
|
||||
++first_array_join;
|
||||
}
|
||||
@ -1093,7 +1093,7 @@ void ExpressionActionsChain::addStep()
|
||||
|
||||
void ExpressionActionsChain::finalize()
|
||||
{
|
||||
/// Финализируем все шаги. Справа налево, чтобы определять ненужные входные столбцы.
|
||||
/// Finalize all steps. Right to left to define unnecessary input columns.
|
||||
for (int i = static_cast<int>(steps.size()) - 1; i >= 0; --i)
|
||||
{
|
||||
Names required_output = steps[i].required_output;
|
||||
@ -1105,7 +1105,7 @@ void ExpressionActionsChain::finalize()
|
||||
steps[i].actions->finalize(required_output);
|
||||
}
|
||||
|
||||
/// Когда возможно, перенесем ARRAY JOIN из более ранних шагов в более поздние.
|
||||
/// When possible, move the ARRAY JOIN from earlier steps to later steps.
|
||||
for (size_t i = 1; i < steps.size(); ++i)
|
||||
{
|
||||
ExpressionAction action;
|
||||
@ -1113,13 +1113,13 @@ void ExpressionActionsChain::finalize()
|
||||
steps[i].actions->prependArrayJoin(action, steps[i - 1].actions->getSampleBlock());
|
||||
}
|
||||
|
||||
/// Добавим выбрасывание ненужных столбцов в начало каждого шага.
|
||||
/// Adding the ejection of unnecessary columns to the beginning of each step.
|
||||
for (size_t i = 1; i < steps.size(); ++i)
|
||||
{
|
||||
size_t columns_from_previous = steps[i - 1].actions->getSampleBlock().columns();
|
||||
|
||||
/// Если на выходе предыдущего шага образуются ненужные столбцы, добавим в начало этого шага их выбрасывание.
|
||||
/// За исключением случая, когда мы выбросим все столбцы и потеряем количество строк в блоке.
|
||||
/// If unnecessary columns are formed at the output of the previous step, we'll add them to the beginning of this step.
|
||||
/// Except when we drop all the columns and lose the number of rows in the block.
|
||||
if (!steps[i].actions->getRequiredColumnsWithTypes().empty()
|
||||
&& columns_from_previous > steps[i].actions->getRequiredColumnsWithTypes().size())
|
||||
steps[i].actions->prependProjectInput();
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -139,7 +139,7 @@ void ExternalDictionaries::reloadImpl(const bool throw_on_error)
|
||||
|
||||
try
|
||||
{
|
||||
/// Если словарь не удалось ни разу загрузить или даже не удалось инициализировать из конфига.
|
||||
/// If the dictionary failed to load or even failed to initialize from the config.
|
||||
if (!dictionary.second.dict)
|
||||
continue;
|
||||
|
||||
@ -250,7 +250,7 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
|
||||
|
||||
auto dict_ptr = DictionaryFactory::instance().create(name, *config, key, context);
|
||||
|
||||
/// Если словарь не удалось загрузить.
|
||||
/// If the dictionary could not be loaded.
|
||||
if (const auto exception_ptr = dict_ptr->getCreationException())
|
||||
{
|
||||
const auto failed_dict_it = failed_dictionaries.find(name);
|
||||
@ -308,8 +308,8 @@ void ExternalDictionaries::reloadFromFile(const std::string & config_path, const
|
||||
{
|
||||
if (!name.empty())
|
||||
{
|
||||
/// Если для словаря не удалось загрузить данные или даже не удалось инициализировать из конфига.
|
||||
/// - всё-равно вставляем информацию в dictionaries, с нулевым указателем dict.
|
||||
/// If the dictionary could not load data or even failed to initialize from the config.
|
||||
/// - all the same we insert information into the `dictionaries`, with the zero pointer `dict`.
|
||||
|
||||
const std::lock_guard<std::mutex> lock{dictionaries_mutex};
|
||||
|
||||
|
@ -25,8 +25,8 @@ namespace ErrorCodes
|
||||
namespace
|
||||
{
|
||||
|
||||
/// Вспомогательная структура для оформления ответа на запрос DESCRIBE TABLE с Distributed-таблицей.
|
||||
/// Содержит информацию про локальную таблицу, которая была получена с одной реплики.
|
||||
/// A helper structure for performing a response to a DESCRIBE TABLE query with a Distributed table.
|
||||
/// Contains information about the local table that was retrieved from a single replica.
|
||||
struct TableDescription
|
||||
{
|
||||
TableDescription(const Block & block, const BlockExtraInfo & extra_info_)
|
||||
@ -145,9 +145,9 @@ BlockIO InterpreterCheckQuery::execute()
|
||||
auto distributed_table = typeid_cast<StorageDistributed *>(&*table);
|
||||
if (distributed_table != nullptr)
|
||||
{
|
||||
/// Для таблиц с движком Distributed запрос CHECK TABLE отправляет запрос DESCRIBE TABLE на все реплики.
|
||||
/// Проверяется идентичность структур (имена столбцов + типы столбцов + типы по-умолчанию + выражения
|
||||
/// по-умолчанию) таблиц, на котороые смотрит распределённая таблица.
|
||||
/// For tables with the Distributed engine, the CHECK TABLE query sends a DESCRIBE TABLE request to all replicas.
|
||||
/// The identity of the structures is checked (column names + column types + default types + expressions
|
||||
/// by default) of the tables that the distributed table looks at.
|
||||
|
||||
const auto settings = context.getSettings();
|
||||
|
||||
@ -161,7 +161,7 @@ BlockIO InterpreterCheckQuery::execute()
|
||||
throw Exception("InterpreterCheckQuery: Internal error", ErrorCodes::LOGICAL_ERROR);
|
||||
auto & stream = *stream_ptr;
|
||||
|
||||
/// Получить все данные от запросов DESCRIBE TABLE.
|
||||
/// Get all data from the DESCRIBE TABLE queries.
|
||||
|
||||
TableDescriptions table_descriptions;
|
||||
|
||||
@ -188,7 +188,7 @@ BlockIO InterpreterCheckQuery::execute()
|
||||
if (table_descriptions.empty())
|
||||
throw Exception("Received empty data", ErrorCodes::RECEIVED_EMPTY_DATA);
|
||||
|
||||
/// Определить класс эквивалентности каждой структуры таблицы.
|
||||
/// Define an equivalence class for each table structure.
|
||||
|
||||
std::sort(table_descriptions.begin(), table_descriptions.end());
|
||||
|
||||
@ -206,7 +206,7 @@ BlockIO InterpreterCheckQuery::execute()
|
||||
prev = it;
|
||||
}
|
||||
|
||||
/// Составить результат.
|
||||
/// Construct the result.
|
||||
|
||||
ColumnPtr status_column = std::make_shared<ColumnUInt8>();
|
||||
ColumnPtr host_name_column = std::make_shared<ColumnString>();
|
||||
@ -216,7 +216,7 @@ BlockIO InterpreterCheckQuery::execute()
|
||||
ColumnPtr structure_class_column = std::make_shared<ColumnUInt32>();
|
||||
ColumnPtr structure_column = std::make_shared<ColumnString>();
|
||||
|
||||
/// Это значение равно 1, если структура нигде не отлчиается, а 0 в противном случае.
|
||||
/// This value is 1 if the structure is not disposed of anywhere, but 0 otherwise.
|
||||
UInt8 status_value = (structure_class == 0) ? 1 : 0;
|
||||
|
||||
for (const auto & desc : table_descriptions)
|
||||
|
@ -151,7 +151,7 @@ void InterpreterCreateQuery::createDatabase(ASTCreateQuery & create)
|
||||
|
||||
using ColumnsAndDefaults = std::pair<NamesAndTypesList, ColumnDefaults>;
|
||||
|
||||
/// AST в список столбцов с типами. Столбцы типа Nested развернуты в список настоящих столбцов.
|
||||
/// AST to the list of columns with types. Columns of Nested type are expanded into a list of real columns.
|
||||
static ColumnsAndDefaults parseColumns(
|
||||
ASTPtr expression_list, const Context & context)
|
||||
{
|
||||
@ -436,7 +436,7 @@ String InterpreterCreateQuery::setEngine(
|
||||
}
|
||||
else if (!create.as_table.empty())
|
||||
{
|
||||
/// NOTE Получение структуры у таблицы, указанной в AS делается не атомарно с созданием таблицы.
|
||||
/// NOTE Getting the structure from the table specified in the AS is done not atomically with the creation of the table.
|
||||
|
||||
String as_database_name = create.as_database.empty() ? context.getCurrentDatabase() : create.as_database;
|
||||
String as_table_name = create.as_table;
|
||||
@ -472,7 +472,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
|
||||
std::unique_ptr<InterpreterSelectQuery> interpreter_select;
|
||||
Block as_select_sample;
|
||||
/// Для таблиц типа view, чтобы получить столбцы, может понадобиться sample_block.
|
||||
/// For `view` type tables, you may need `sample_block` to get the columns.
|
||||
if (create.select && (!create.attach || (!create.columns && (create.is_view || create.is_materialized_view))))
|
||||
{
|
||||
interpreter_select = std::make_unique<InterpreterSelectQuery>(create.select, context);
|
||||
@ -493,7 +493,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
/// Set and retrieve list of columns.
|
||||
ColumnsInfo columns = setColumns(create, as_select_sample, as_storage);
|
||||
|
||||
/// Выбор нужного движка таблицы
|
||||
/// Select the desired table engine
|
||||
String storage_name = setEngine(create, as_storage);
|
||||
|
||||
StoragePtr res;
|
||||
@ -505,10 +505,10 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
{
|
||||
context.assertDatabaseExists(database_name);
|
||||
|
||||
/** Если таблица уже существует, и в запросе указано IF NOT EXISTS,
|
||||
* то мы разрешаем конкуррентные запросы CREATE (которые ничего не делают).
|
||||
* Иначе конкуррентные запросы на создание таблицы, если таблицы не существует,
|
||||
* могут кидать исключение, даже если указано IF NOT EXISTS.
|
||||
/** If the table already exists, and the request specifies IF NOT EXISTS,
|
||||
* then we allow concurrent CREATE queries (which do nothing).
|
||||
* Otherwise, concurrent queries for creating a table, if the table does not exist,
|
||||
* can throw an exception, even if IF NOT EXISTS is specified.
|
||||
*/
|
||||
guard = context.getDDLGuardIfTableDoesntExist(database_name, table_name,
|
||||
"Table " + database_name + "." + table_name + " is creating or attaching right now");
|
||||
@ -533,12 +533,12 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
|
||||
context.getDatabase(database_name)->createTable(table_name, res, query_ptr, storage_name, context.getSettingsRef());
|
||||
}
|
||||
|
||||
/// Если запрос CREATE SELECT, то вставим в таблицу данные
|
||||
/// If the CREATE SELECT query is, insert the data into the table
|
||||
if (create.select && storage_name != "View" && (storage_name != "MaterializedView" || create.is_populate))
|
||||
{
|
||||
auto table_lock = res->lockStructure(true);
|
||||
|
||||
/// Также см. InterpreterInsertQuery.
|
||||
/// Also see InterpreterInsertQuery.
|
||||
BlockOutputStreamPtr out =
|
||||
std::make_shared<ProhibitColumnsBlockOutputStream>(
|
||||
std::make_shared<AddingDefaultBlockOutputStream>(
|
||||
|
@ -118,21 +118,21 @@ BlockIO InterpreterDropQuery::execute()
|
||||
|
||||
if (drop_database)
|
||||
{
|
||||
/// Удаление базы данных. Таблицы в ней уже удалены.
|
||||
/// Delete the database. The tables in it have already been deleted.
|
||||
|
||||
auto lock = context.getLock();
|
||||
|
||||
/// Кто-то мог успеть удалить БД до нас.
|
||||
/// Someone could have time to delete the database before us.
|
||||
context.assertDatabaseExists(database_name);
|
||||
|
||||
/// Кто-то мог успеть создать таблицу в удаляемой БД, пока мы удаляли таблицы без лока контекста.
|
||||
/// Someone could have time to create a table in the database to be deleted while we deleted the tables without the context lock.
|
||||
if (!context.getDatabase(database_name)->empty())
|
||||
throw Exception("New table appeared in database being dropped. Try dropping it again.", ErrorCodes::DATABASE_NOT_EMPTY);
|
||||
|
||||
/// Удаляем информацию о БД из оперативки
|
||||
/// Delete database information from the RAM
|
||||
auto database = context.detachDatabase(database_name);
|
||||
|
||||
/// Удаляем БД.
|
||||
/// Delete the database.
|
||||
database->drop();
|
||||
|
||||
Poco::File(data_path).remove(false);
|
||||
|
@ -84,7 +84,7 @@ std::unique_ptr<IInterpreter> InterpreterFactory::get(ASTPtr & query, Context &
|
||||
}
|
||||
else if (typeid_cast<ASTSetQuery *>(query.get()))
|
||||
{
|
||||
/// readonly проверяется внутри InterpreterSetQuery
|
||||
/// readonly is checked inside InterpreterSetQuery
|
||||
return std::make_unique<InterpreterSetQuery>(query, context);
|
||||
}
|
||||
else if (typeid_cast<ASTOptimizeQuery *>(query.get()))
|
||||
|
@ -43,7 +43,7 @@ StoragePtr InterpreterInsertQuery::getTable()
|
||||
{
|
||||
ASTInsertQuery & query = typeid_cast<ASTInsertQuery &>(*query_ptr);
|
||||
|
||||
/// В какую таблицу писать.
|
||||
/// In what table to write.
|
||||
return context.getTable(query.database, query.table);
|
||||
}
|
||||
|
||||
@ -51,19 +51,19 @@ Block InterpreterInsertQuery::getSampleBlock()
|
||||
{
|
||||
ASTInsertQuery & query = typeid_cast<ASTInsertQuery &>(*query_ptr);
|
||||
|
||||
/// Если в запросе не указана информация о столбцах
|
||||
/// If the query does not include information about columns
|
||||
if (!query.columns)
|
||||
return getTable()->getSampleBlockNonMaterialized();
|
||||
|
||||
Block table_sample = getTable()->getSampleBlock();
|
||||
|
||||
/// Формируем блок, основываясь на именах столбцов из запроса
|
||||
/// Form the block based on the column names from the query
|
||||
Block res;
|
||||
for (const auto & identifier : query.columns->children)
|
||||
{
|
||||
std::string current_name = identifier->getColumnName();
|
||||
|
||||
/// В таблице нет столбца с таким именем
|
||||
/// The table does not have a column with that name
|
||||
if (!table_sample.has(current_name))
|
||||
throw Exception("No such column " + current_name + " in table " + query.table, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE);
|
||||
|
||||
@ -87,7 +87,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
|
||||
NamesAndTypesListPtr required_columns = std::make_shared<NamesAndTypesList>(table->getColumnsList());
|
||||
|
||||
/// Создаем конвейер из нескольких стримов, в которые будем писать данные.
|
||||
/// We create a pipeline of several streams, into which we will write data.
|
||||
BlockOutputStreamPtr out;
|
||||
|
||||
out = std::make_shared<PushingToViewsBlockOutputStream>(query.database, query.table, context, query_ptr);
|
||||
@ -110,7 +110,7 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
BlockIO res;
|
||||
res.out_sample = getSampleBlock();
|
||||
|
||||
/// Какой тип запроса: INSERT или INSERT SELECT?
|
||||
/// What type of query: INSERT or INSERT SELECT?
|
||||
if (!query.select)
|
||||
{
|
||||
res.out = out;
|
||||
|
@ -78,7 +78,7 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi
|
||||
|
||||
if (is_first_select_inside_union_all)
|
||||
{
|
||||
/// Создать цепочку запросов SELECT.
|
||||
/// Create a SELECT query chain.
|
||||
InterpreterSelectQuery * interpreter = this;
|
||||
ASTPtr tail = query.next_union_all;
|
||||
|
||||
@ -98,15 +98,15 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi
|
||||
{
|
||||
basicInit(input);
|
||||
|
||||
// Мы выполняем этот код именно здесь, потому что в противном случае следующего рода запрос бы не срабатывал:
|
||||
// We execute this code here, because otherwise the following kind of query would not work
|
||||
// SELECT X FROM (SELECT * FROM (SELECT 1 AS X, 2 AS Y) UNION ALL SELECT 3, 4)
|
||||
// из-за того, что астериски заменены столбцами только при создании объектов query_analyzer в basicInit().
|
||||
// because the asterisk is replaced with columns only when query_analyzer objects are created in basicInit().
|
||||
renameColumns();
|
||||
|
||||
if (!required_column_names.empty() && (table_column_names.size() != required_column_names.size()))
|
||||
{
|
||||
rewriteExpressionList(required_column_names);
|
||||
/// Теперь имеется устаревшая информация для выполнения запроса. Обновляем эту информацию.
|
||||
/// Now there is obsolete information to execute the query. We update this information.
|
||||
initQueryAnalyzer();
|
||||
}
|
||||
}
|
||||
@ -135,9 +135,9 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
|
||||
{
|
||||
if (query_table && typeid_cast<const ASTFunction *>(query_table.get()))
|
||||
{
|
||||
/// Получить табличную функцию
|
||||
/// Get the table function
|
||||
TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast<const ASTFunction *>(query_table.get())->name, context);
|
||||
/// Выполнить ее и запомнить результат
|
||||
/// Run it and remember the result
|
||||
storage = table_function_ptr->execute(query_table, context);
|
||||
}
|
||||
else
|
||||
@ -160,7 +160,7 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
|
||||
|
||||
query_analyzer = std::make_unique<ExpressionAnalyzer>(query_ptr, context, storage, table_column_names, subquery_depth, !only_analyze);
|
||||
|
||||
/// Сохраняем в query context новые временные таблицы
|
||||
/// Save the new temporary tables in the query context
|
||||
for (auto & it : query_analyzer->getExternalTables())
|
||||
if (!context.tryGetExternalTable(it.first))
|
||||
context.addExternalTable(it.first, it.second);
|
||||
@ -170,7 +170,7 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_)
|
||||
|
||||
if (is_first_select_inside_union_all)
|
||||
{
|
||||
/// Проверяем, что результаты всех запросов SELECT cовместимые.
|
||||
/// We check that the results of all SELECT queries are compatible.
|
||||
Block first = getSampleBlock();
|
||||
for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get())
|
||||
{
|
||||
@ -294,8 +294,8 @@ void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, St
|
||||
auto query_database = query.database();
|
||||
auto query_table = query.table();
|
||||
|
||||
/** Если таблица не указана - используем таблицу system.one.
|
||||
* Если база данных не указана - используем текущую базу данных.
|
||||
/** If the table is not specified - use the table `system.one`.
|
||||
* If the database is not specified - use the current database.
|
||||
*/
|
||||
if (query_database)
|
||||
database_name = typeid_cast<ASTIdentifier &>(*query_database).name;
|
||||
@ -331,8 +331,8 @@ DataTypes InterpreterSelectQuery::getReturnTypes()
|
||||
Block InterpreterSelectQuery::getSampleBlock()
|
||||
{
|
||||
Block block = query_analyzer->getSelectSampleBlock();
|
||||
/// создадим ненулевые колонки, чтобы SampleBlock можно было
|
||||
/// писать (читать) с помощью BlockOut(In)putStream'ов
|
||||
/// create non-zero columns so that SampleBlock can be
|
||||
/// written (read) with BlockOut(In)putStreams
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
ColumnWithTypeAndName & col = block.safeGetByPosition(i);
|
||||
@ -362,10 +362,10 @@ BlockIO InterpreterSelectQuery::execute()
|
||||
|
||||
executeUnion();
|
||||
|
||||
/// Ограничения на результат, квота на результат, а также колбек для прогресса.
|
||||
/// Constraints on the result, the quota on the result, and also callback for progress.
|
||||
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(streams[0].get()))
|
||||
{
|
||||
/// Ограничения действуют только на конечный результат.
|
||||
/// Constraints apply only to the final result.
|
||||
if (to_stage == QueryProcessingStage::Complete)
|
||||
{
|
||||
IProfilingBlockInputStream::LocalLimits limits;
|
||||
@ -411,21 +411,21 @@ const BlockInputStreams & InterpreterSelectQuery::executeWithoutUnion()
|
||||
|
||||
void InterpreterSelectQuery::executeSingleQuery()
|
||||
{
|
||||
/** Потоки данных. При параллельном выполнении запроса, имеем несколько потоков данных.
|
||||
* Если нет GROUP BY, то выполним все операции до ORDER BY и LIMIT параллельно, затем
|
||||
* если есть ORDER BY, то склеим потоки с помощью UnionBlockInputStream, а затем MergеSortingBlockInputStream,
|
||||
* если нет, то склеим с помощью UnionBlockInputStream,
|
||||
* затем применим LIMIT.
|
||||
* Если есть GROUP BY, то выполним все операции до GROUP BY, включительно, параллельно;
|
||||
* параллельный GROUP BY склеит потоки в один,
|
||||
* затем выполним остальные операции с одним получившимся потоком.
|
||||
* Если запрос является членом цепочки UNION ALL и не содержит GROUP BY, ORDER BY, DISTINCT, или LIMIT,
|
||||
* то объединение источников данных выполняется не на этом уровне, а на верхнем уровне.
|
||||
/** Streams of data. When the query is executed in parallel, we have several data streams.
|
||||
* If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then
|
||||
* if there is an ORDER BY, then glue the streams using UnionBlockInputStream, and then MergeSortingBlockInputStream,
|
||||
* if not, then glue it using UnionBlockInputStream,
|
||||
* then apply LIMIT.
|
||||
* If there is GROUP BY, then we will perform all operations up to GROUP BY, inclusive, in parallel;
|
||||
* a parallel GROUP BY will glue streams into one,
|
||||
* then perform the remaining operations with one resulting stream.
|
||||
* If the query is a member of the UNION ALL chain and does not contain GROUP BY, ORDER BY, DISTINCT, or LIMIT,
|
||||
* then the data sources are merged not at this level, but at the upper level.
|
||||
*/
|
||||
|
||||
union_within_single_query = false;
|
||||
|
||||
/** Вынем данные из Storage. from_stage - до какой стадии запрос был выполнен в Storage. */
|
||||
/** Take out the data from Storage. from_stage - to what stage the request was completed in Storage. */
|
||||
QueryProcessingStage::Enum from_stage = executeFetchColumns();
|
||||
|
||||
LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(to_stage));
|
||||
@ -438,26 +438,26 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
bool has_having = false;
|
||||
bool has_order_by = false;
|
||||
|
||||
ExpressionActionsPtr before_join; /// включая JOIN
|
||||
ExpressionActionsPtr before_join; /// including JOIN
|
||||
ExpressionActionsPtr before_where;
|
||||
ExpressionActionsPtr before_aggregation;
|
||||
ExpressionActionsPtr before_having;
|
||||
ExpressionActionsPtr before_order_and_select;
|
||||
ExpressionActionsPtr final_projection;
|
||||
|
||||
/// Столбцы из списка SELECT, до переименования в алиасы.
|
||||
/// Columns from the SELECT list, before renaming them to aliases.
|
||||
Names selected_columns;
|
||||
|
||||
/// Нужно ли выполнять первую часть конвейера - выполняемую на удаленных серверах при распределенной обработке.
|
||||
/// Do I need to perform the first part of the pipeline - running on remote servers during distributed processing.
|
||||
bool first_stage = from_stage < QueryProcessingStage::WithMergeableState
|
||||
&& to_stage >= QueryProcessingStage::WithMergeableState;
|
||||
/// Нужно ли выполнять вторую часть конвейера - выполняемую на сервере-инициаторе при распределенной обработке.
|
||||
/// Do I need to execute the second part of the pipeline - running on the initiating server during distributed processing.
|
||||
bool second_stage = from_stage <= QueryProcessingStage::WithMergeableState
|
||||
&& to_stage > QueryProcessingStage::WithMergeableState;
|
||||
|
||||
/** Сначала составим цепочку действий и запомним нужные шаги из нее.
|
||||
* Независимо от from_stage и to_stage составим полную последовательность действий, чтобы выполнять оптимизации и
|
||||
* выбрасывать ненужные столбцы с учетом всего запроса. В ненужных частях запроса не будем выполнять подзапросы.
|
||||
/** First we compose a chain of actions and remember the necessary steps from it.
|
||||
* Regardless of from_stage and to_stage, we will compose a complete sequence of actions to perform optimization and
|
||||
* throw out unnecessary columns based on the entire query. In unnecessary parts of the query, we will not execute subqueries.
|
||||
*/
|
||||
|
||||
{
|
||||
@ -502,7 +502,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
}
|
||||
}
|
||||
|
||||
/// Если есть агрегация, выполняем выражения в SELECT и ORDER BY на инициировавшем сервере, иначе - на серверах-источниках.
|
||||
/// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers.
|
||||
query_analyzer->appendSelect(chain, need_aggregate ? !second_stage : !first_stage);
|
||||
selected_columns = chain.getLastStep().required_output;
|
||||
has_order_by = query_analyzer->appendOrderBy(chain, need_aggregate ? !second_stage : !first_stage);
|
||||
@ -516,23 +516,23 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
chain.clear();
|
||||
}
|
||||
|
||||
/** Если данных нет.
|
||||
* Эта проверка специально вынесена чуть ниже, чем она могла бы быть (сразу после executeFetchColumns),
|
||||
* чтобы запрос был проанализирован, и в нём могли бы быть обнаружены ошибки (например, несоответствия типов).
|
||||
* Иначе мог бы вернуться пустой результат на некорректный запрос.
|
||||
/** If there is no data.
|
||||
* This check is specially postponed slightly lower than it could be (immediately after executeFetchColumns),
|
||||
* for the query to be analyzed, and errors (for example, type mismatches) could be found in it.
|
||||
* Otherwise, the empty result could be returned for the incorrect query.
|
||||
*/
|
||||
if (hasNoData())
|
||||
return;
|
||||
|
||||
/// Перед выполнением WHERE и HAVING уберем из блока лишние столбцы (в основном, ключи агрегации).
|
||||
/// Before executing WHERE and HAVING, remove the extra columns from the block (mostly the aggregation keys).
|
||||
if (has_where)
|
||||
before_where->prependProjectInput();
|
||||
if (has_having)
|
||||
before_having->prependProjectInput();
|
||||
|
||||
/// Теперь составим потоки блоков, выполняющие нужные действия.
|
||||
/// Now we will compose block streams that perform the necessary actions.
|
||||
|
||||
/// Нужно ли агрегировать в отдельную строку строки, не прошедшие max_rows_to_group_by.
|
||||
/// Do I need to aggregate in a separate row rows that have not passed max_rows_to_group_by.
|
||||
bool aggregate_overflow_row =
|
||||
need_aggregate &&
|
||||
query.group_by_with_totals &&
|
||||
@ -540,7 +540,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
settings.limits.group_by_overflow_mode == OverflowMode::ANY &&
|
||||
settings.totals_mode != TotalsMode::AFTER_HAVING_EXCLUSIVE;
|
||||
|
||||
/// Нужно ли после агрегации сразу финализировать агрегатные функции.
|
||||
/// Do I need to immediately finalize the aggregate functions after the aggregation?
|
||||
bool aggregate_final =
|
||||
need_aggregate &&
|
||||
to_stage > QueryProcessingStage::WithMergeableState &&
|
||||
@ -549,7 +549,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
if (first_stage)
|
||||
{
|
||||
if (has_join)
|
||||
for (auto & stream : streams) /// Применяем ко всем источникам кроме stream_with_non_joined_data.
|
||||
for (auto & stream : streams) /// Applies to all sources except stream_with_non_joined_data.
|
||||
stream = std::make_shared<ExpressionBlockInputStream>(stream, before_join);
|
||||
|
||||
if (has_where)
|
||||
@ -563,10 +563,10 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
executeDistinct(true, selected_columns);
|
||||
}
|
||||
|
||||
/** При распределённой обработке запроса,
|
||||
* если не указаны GROUP, HAVING,
|
||||
* но есть ORDER или LIMIT,
|
||||
* то выполним предварительную сортировку и LIMIT на удалёном сервере.
|
||||
/** For distributed query processing,
|
||||
* if no GROUP, HAVING set,
|
||||
* but there is an ORDER or LIMIT,
|
||||
* then we will perform the preliminary sorting and LIMIT on the remote server.
|
||||
*/
|
||||
if (!second_stage && !need_aggregate && !has_having)
|
||||
{
|
||||
@ -587,7 +587,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
|
||||
if (need_aggregate)
|
||||
{
|
||||
/// Если нужно объединить агрегированные результаты с нескольких серверов
|
||||
/// If you need to combine aggregated results from multiple servers
|
||||
if (!first_stage)
|
||||
executeMergeAggregated(aggregate_overflow_row, aggregate_final);
|
||||
|
||||
@ -611,19 +611,19 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
|
||||
if (has_order_by)
|
||||
{
|
||||
/** Если при распределённой обработке запроса есть ORDER BY,
|
||||
* но нет агрегации, то на удалённых серверах был сделан ORDER BY
|
||||
* - поэтому, делаем merge сортированных потоков с удалённых серверов.
|
||||
/** If there is an ORDER BY for distributed query processing,
|
||||
* but there is no aggregation, then on the remote servers ORDER BY was made
|
||||
* - therefore, we merge the sorted streams from remote servers.
|
||||
*/
|
||||
if (!first_stage && !need_aggregate && !(query.group_by_with_totals && !aggregate_final))
|
||||
executeMergeSorted();
|
||||
else /// Иначе просто сортировка.
|
||||
else /// Otherwise, just sort.
|
||||
executeOrder();
|
||||
}
|
||||
|
||||
executeProjection(final_projection);
|
||||
|
||||
/// На этой стадии можно считать минимумы и максимумы, если надо.
|
||||
/// At this stage, we can calculate the minimums and maximums, if necessary.
|
||||
if (settings.extremes)
|
||||
{
|
||||
transformStreams([&](auto & stream)
|
||||
@ -633,8 +633,8 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
});
|
||||
}
|
||||
|
||||
/** Оптимизация - если источников несколько и есть LIMIT, то сначала применим предварительный LIMIT,
|
||||
* ограничивающий число записей в каждом до offset + limit.
|
||||
/** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT,
|
||||
* limiting the number of entries in each up to `offset + limit`.
|
||||
*/
|
||||
if (query.limit_length && hasMoreThanOneStream() && !query.distinct && !query.limit_by_expression_list)
|
||||
executePreLimit();
|
||||
@ -663,7 +663,7 @@ void InterpreterSelectQuery::executeSingleQuery()
|
||||
}
|
||||
}
|
||||
|
||||
/** Если данных нет. */
|
||||
/** If there is no data. */
|
||||
if (hasNoData())
|
||||
return;
|
||||
|
||||
@ -690,14 +690,14 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
if (!hasNoData())
|
||||
return QueryProcessingStage::FetchColumns;
|
||||
|
||||
/// Интерпретатор подзапроса, если подзапрос
|
||||
/// The subquery interpreter, if the subquery
|
||||
std::experimental::optional<InterpreterSelectQuery> interpreter_subquery;
|
||||
|
||||
/// Список столбцов, которых нужно прочитать, чтобы выполнить запрос.
|
||||
/// List of columns to read to execute the query.
|
||||
Names required_columns = query_analyzer->getRequiredColumns();
|
||||
/// Действия для вычисления ALIAS, если потребуется.
|
||||
/// Actions to calculate ALIAS if required.
|
||||
ExpressionActionsPtr alias_actions;
|
||||
/// Требуются ли ALIAS столбцы для выполнения запроса?
|
||||
/// Are ALIAS columns required for query execution?
|
||||
auto alias_columns_required = false;
|
||||
|
||||
if (storage && !storage->alias_columns.empty())
|
||||
@ -714,7 +714,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
|
||||
if (alias_columns_required)
|
||||
{
|
||||
/// Составим выражение для возврата всех запрошенных столбцов, с вычислением требуемых ALIAS столбцов.
|
||||
/// We will create an expression to return all the requested columns, with the calculation of the required ALIAS columns.
|
||||
auto required_columns_expr_list = std::make_shared<ASTExpressionList>();
|
||||
|
||||
for (const auto & column : required_columns)
|
||||
@ -728,7 +728,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
|
||||
alias_actions = ExpressionAnalyzer{required_columns_expr_list, context, storage, table_column_names}.getActions(true);
|
||||
|
||||
/// Множество требуемых столбцов могло быть дополнено в результате добавления действия для вычисления ALIAS.
|
||||
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
|
||||
required_columns = alias_actions->getRequiredColumns();
|
||||
}
|
||||
}
|
||||
@ -736,21 +736,21 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
auto query_table = query.table();
|
||||
if (query_table && typeid_cast<ASTSelectQuery *>(query_table.get()))
|
||||
{
|
||||
/** Для подзапроса не действуют ограничения на максимальный размер результата.
|
||||
* Так как результат поздапроса - ещё не результат всего запроса.
|
||||
/** There are no limits on the maximum size of the result for the subquery.
|
||||
* Since the result of the query is not the result of the entire query.
|
||||
*/
|
||||
Context subquery_context = context;
|
||||
Settings subquery_settings = context.getSettings();
|
||||
subquery_settings.limits.max_result_rows = 0;
|
||||
subquery_settings.limits.max_result_bytes = 0;
|
||||
/// Вычисление extremes не имеет смысла и не нужно (если его делать, то в результате всего запроса могут взяться extremes подзапроса).
|
||||
/// The calculation of extremes does not make sense and is not necessary (if you do it, then the extremes of the subquery can be taken for whole query).
|
||||
subquery_settings.extremes = 0;
|
||||
subquery_context.setSettings(subquery_settings);
|
||||
|
||||
interpreter_subquery.emplace(
|
||||
query_table, subquery_context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1);
|
||||
|
||||
/// Если во внешнем запросе есть аггрегация, то WITH TOTALS игнорируется в подзапросе.
|
||||
/// If there is an aggregation in the outer query, WITH TOTALS is ignored in the subquery.
|
||||
if (query_analyzer->hasAggregation())
|
||||
interpreter_subquery->ignoreWithTotals();
|
||||
}
|
||||
@ -764,16 +764,16 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
if (query.prewhere_expression && (!storage || !storage->supportsPrewhere()))
|
||||
throw Exception(storage ? "Storage " + storage->getName() + " doesn't support PREWHERE" : "Illegal PREWHERE", ErrorCodes::ILLEGAL_PREWHERE);
|
||||
|
||||
/** При распределённой обработке запроса, в потоках почти не делается вычислений,
|
||||
* а делается ожидание и получение данных с удалённых серверов.
|
||||
* Если у нас 20 удалённых серверов, а max_threads = 8, то было бы не очень хорошо
|
||||
* соединяться и опрашивать только по 8 серверов одновременно.
|
||||
* Чтобы одновременно опрашивалось больше удалённых серверов,
|
||||
* вместо max_threads используется max_distributed_connections.
|
||||
/** With distributed query processing, almost no computations are done in the threads,
|
||||
* but wait and receive data from remote servers.
|
||||
* If we have 20 remote servers, and max_threads = 8, then it would not be very good
|
||||
* connect and ask only 8 servers at a time.
|
||||
* To simultaneously query more remote servers,
|
||||
* instead of max_threads, max_distributed_connections is used.
|
||||
*
|
||||
* Сохраним изначальное значение max_threads в settings_for_storage
|
||||
* - эти настройки будут переданы на удалённые серверы при распределённой обработке запроса,
|
||||
* и там должно быть оригинальное значение max_threads, а не увеличенное.
|
||||
* Save the initial value of max_threads in settings_for_storage
|
||||
* - these settings will be passed to remote servers for distributed query processing,
|
||||
* and there must be an original value of max_threads, not an increased value.
|
||||
*/
|
||||
bool is_remote = false;
|
||||
Settings settings_for_storage = settings;
|
||||
@ -783,7 +783,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
settings.max_threads = settings.max_distributed_connections;
|
||||
}
|
||||
|
||||
/// Ограничение на количество столбцов для чтения.
|
||||
/// Limitation on the number of columns to read.
|
||||
if (settings.limits.max_columns_to_read && required_columns.size() > settings.limits.max_columns_to_read)
|
||||
throw Exception("Limit for number of columns to read exceeded. "
|
||||
"Requested: " + toString(required_columns.size())
|
||||
@ -794,9 +794,9 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
size_t limit_offset = 0;
|
||||
getLimitLengthAndOffset(query, limit_length, limit_offset);
|
||||
|
||||
/** Оптимизация - если не указаны DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY но указан LIMIT, и limit + offset < max_block_size,
|
||||
* то в качестве размера блока будем использовать limit + offset (чтобы не читать из таблицы больше, чем запрошено),
|
||||
* а также установим количество потоков в 1.
|
||||
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY 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.prewhere_expression
|
||||
@ -817,7 +817,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
|
||||
query_analyzer->makeSetsForIndex();
|
||||
|
||||
/// Инициализируем изначальные потоки данных, на которые накладываются преобразования запроса. Таблица или подзапрос?
|
||||
/// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery?
|
||||
if (!interpreter_subquery)
|
||||
{
|
||||
size_t max_streams = settings.max_threads;
|
||||
@ -825,14 +825,14 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
if (max_streams == 0)
|
||||
throw Exception("Logical error: zero number of streams requested", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
/// Если надо - запрашиваем больше источников, чем количество потоков - для более равномерного распределения работы по потокам.
|
||||
/// If necessary, we request more sources than the number of threads - to distribute the work evenly over the threads.
|
||||
if (max_streams > 1 && !is_remote)
|
||||
max_streams *= settings.max_streams_to_max_threads_ratio;
|
||||
|
||||
ASTPtr actual_query_ptr;
|
||||
if (storage->isRemote())
|
||||
{
|
||||
/// В случае удаленного запроса отправляем только SELECT, который выполнится.
|
||||
/// In case of a remote query, we send only SELECT, which will be executed.
|
||||
actual_query_ptr = query.cloneFirstSelect();
|
||||
}
|
||||
else
|
||||
@ -843,7 +843,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
settings.max_block_size, max_streams);
|
||||
|
||||
if (alias_actions)
|
||||
/// Обернем каждый поток, возвращенный из таблицы, с целью вычисления и добавления ALIAS столбцов
|
||||
/// Wrap each stream returned from the table to calculate and add ALIAS columns
|
||||
transformStreams([&] (auto & stream)
|
||||
{
|
||||
stream = std::make_shared<ExpressionBlockInputStream>(stream, alias_actions);
|
||||
@ -860,9 +860,9 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
|
||||
streams.insert(streams.end(), subquery_streams.begin(), subquery_streams.end());
|
||||
}
|
||||
|
||||
/** Установка ограничений и квоты на чтение данных, скорость и время выполнения запроса.
|
||||
* Такие ограничения проверяются на сервере-инициаторе запроса, а не на удалённых серверах.
|
||||
* Потому что сервер-инициатор имеет суммарные данные о выполнении запроса на всех серверах.
|
||||
/** Set the limits and quota for reading data, the speed and time of the query.
|
||||
* Such restrictions are checked on the initiating server of the request, and not on remote servers.
|
||||
* Because the initiating server has a summary of the execution of the request on all servers.
|
||||
*/
|
||||
if (storage && to_stage == QueryProcessingStage::Complete)
|
||||
{
|
||||
@ -912,9 +912,9 @@ void InterpreterSelectQuery::executeAggregation(ExpressionActionsPtr expression,
|
||||
AggregateDescriptions aggregates;
|
||||
query_analyzer->getAggregateInfo(key_names, aggregates);
|
||||
|
||||
/** Двухуровневая агрегация полезна в двух случаях:
|
||||
* 1. Делается параллельная агрегация, и результаты надо параллельно мерджить.
|
||||
* 2. Делается агрегация с сохранением временных данных на диск, и их нужно мерджить эффективно по памяти.
|
||||
/** Two-level aggregation is useful in two cases:
|
||||
* 1. Parallel aggregation is done, and the results should be measured in parallel.
|
||||
* 2. An aggregation is done with store of temporary data on the disk, and they need to be merged memory efficient.
|
||||
*/
|
||||
bool allow_to_use_two_level_group_by = streams.size() > 1 || settings.limits.max_bytes_before_external_group_by != 0;
|
||||
|
||||
@ -925,7 +925,7 @@ void InterpreterSelectQuery::executeAggregation(ExpressionActionsPtr expression,
|
||||
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0),
|
||||
settings.limits.max_bytes_before_external_group_by, context.getTemporaryPath());
|
||||
|
||||
/// Если источников несколько, то выполняем параллельную агрегацию
|
||||
/// If there are several sources, then we perform parallel aggregation
|
||||
if (streams.size() > 1)
|
||||
{
|
||||
streams[0] = std::make_shared<ParallelAggregatingBlockInputStream>(
|
||||
@ -962,29 +962,29 @@ void InterpreterSelectQuery::executeMergeAggregated(bool overflow_row, bool fina
|
||||
AggregateDescriptions aggregates;
|
||||
query_analyzer->getAggregateInfo(key_names, aggregates);
|
||||
|
||||
/** Есть два режима распределённой агрегации.
|
||||
/** There are two modes of distributed aggregation.
|
||||
*
|
||||
* 1. В разных потоках читать из удалённых серверов блоки.
|
||||
* Сохранить все блоки в оперативку. Объединить блоки.
|
||||
* Если агрегация двухуровневая - распараллелить по номерам корзин.
|
||||
* 1. In different threads read from the remote servers blocks.
|
||||
* Save all the blocks in the RAM. Merge blocks.
|
||||
* If the aggregation is two-level - parallelize to the number of buckets.
|
||||
*
|
||||
* 2. В одном потоке читать по очереди блоки с разных серверов.
|
||||
* В оперативке хранится только по одному блоку с каждого сервера.
|
||||
* Если агрегация двухуровневая - последовательно объединяем блоки каждого следующего уровня.
|
||||
* 2. In one thread, read blocks from different servers in order.
|
||||
* RAM stores only one block from each server.
|
||||
* If the aggregation is a two-level aggregation, we consistently merge the blocks of each next level.
|
||||
*
|
||||
* Второй вариант расходует меньше памяти (до 256 раз меньше)
|
||||
* в случае двухуровневой агрегации, которая используется для больших результатов после GROUP BY,
|
||||
* но при этом может работать медленнее.
|
||||
* The second option consumes less memory (up to 256 times less)
|
||||
* in the case of two-level aggregation, which is used for large results after GROUP BY,
|
||||
* but it can work more slowly.
|
||||
*/
|
||||
|
||||
Aggregator::Params params(key_names, aggregates, overflow_row);
|
||||
|
||||
if (!settings.distributed_aggregation_memory_efficient)
|
||||
{
|
||||
/// Склеим несколько источников в один, распараллеливая работу.
|
||||
/// We union several sources into one, parallelizing the work.
|
||||
executeUnion();
|
||||
|
||||
/// Теперь объединим агрегированные блоки
|
||||
/// Now merge the aggregated blocks
|
||||
streams[0] = std::make_shared<MergingAggregatedBlockInputStream>(streams[0], params, final, original_max_threads);
|
||||
}
|
||||
else
|
||||
@ -1072,7 +1072,7 @@ void InterpreterSelectQuery::executeOrder()
|
||||
{
|
||||
auto sorting_stream = std::make_shared<PartialSortingBlockInputStream>(stream, order_descr, limit);
|
||||
|
||||
/// Ограничения на сортировку
|
||||
/// Limits on sorting
|
||||
IProfilingBlockInputStream::LocalLimits limits;
|
||||
limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL;
|
||||
limits.max_rows_to_read = settings.limits.max_rows_to_sort;
|
||||
@ -1083,10 +1083,10 @@ void InterpreterSelectQuery::executeOrder()
|
||||
stream = sorting_stream;
|
||||
});
|
||||
|
||||
/// Если потоков несколько, то объединяем их в один
|
||||
/// If there are several streams, we merge them into one
|
||||
executeUnion();
|
||||
|
||||
/// Сливаем сортированные блоки.
|
||||
/// Merge the sorted blocks.
|
||||
streams[0] = std::make_shared<MergeSortingBlockInputStream>(
|
||||
streams[0], order_descr, settings.max_block_size, limit,
|
||||
settings.limits.max_bytes_before_external_sort, context.getTemporaryPath());
|
||||
@ -1098,18 +1098,18 @@ void InterpreterSelectQuery::executeMergeSorted()
|
||||
SortDescription order_descr = getSortDescription(query);
|
||||
size_t limit = getLimitForSorting(query);
|
||||
|
||||
/// Если потоков несколько, то объединяем их в один
|
||||
/// If there are several streams, then we merge them into one
|
||||
if (hasMoreThanOneStream())
|
||||
{
|
||||
/** MergingSortedBlockInputStream читает источники последовательно.
|
||||
* Чтобы данные на удалённых серверах готовились параллельно, оборачиваем в AsynchronousBlockInputStream.
|
||||
/** MergingSortedBlockInputStream reads the sources sequentially.
|
||||
* To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream.
|
||||
*/
|
||||
transformStreams([&](auto & stream)
|
||||
{
|
||||
stream = std::make_shared<AsynchronousBlockInputStream>(stream);
|
||||
});
|
||||
|
||||
/// Сливаем сортированные источники в один сортированный источник.
|
||||
/// Merge the sorted sources into one sorted source.
|
||||
streams[0] = std::make_shared<MergingSortedBlockInputStream>(streams, order_descr, settings.max_block_size, limit);
|
||||
streams.resize(1);
|
||||
}
|
||||
@ -1135,7 +1135,7 @@ void InterpreterSelectQuery::executeDistinct(bool before_order, Names columns)
|
||||
|
||||
size_t limit_for_distinct = 0;
|
||||
|
||||
/// Если после этой стадии DISTINCT не будет выполняться ORDER BY, то можно достать не более limit_length + limit_offset различных строк.
|
||||
/// 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.order_expression_list || !before_order)
|
||||
limit_for_distinct = limit_length + limit_offset;
|
||||
|
||||
@ -1152,7 +1152,7 @@ void InterpreterSelectQuery::executeDistinct(bool before_order, Names columns)
|
||||
|
||||
void InterpreterSelectQuery::executeUnion()
|
||||
{
|
||||
/// Если до сих пор есть несколько потоков, то объединяем их в один
|
||||
/// If there are still several streams, then we combine them into one
|
||||
if (hasMoreThanOneStream())
|
||||
{
|
||||
streams[0] = std::make_shared<UnionBlockInputStream<>>(streams, stream_with_non_joined_data, settings.max_threads);
|
||||
@ -1169,14 +1169,14 @@ void InterpreterSelectQuery::executeUnion()
|
||||
}
|
||||
|
||||
|
||||
/// Предварительный LIMIT - применяется в каждом источнике, если источников несколько, до их объединения.
|
||||
/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined.
|
||||
void InterpreterSelectQuery::executePreLimit()
|
||||
{
|
||||
size_t limit_length = 0;
|
||||
size_t limit_offset = 0;
|
||||
getLimitLengthAndOffset(query, limit_length, limit_offset);
|
||||
|
||||
/// Если есть LIMIT
|
||||
/// If there is LIMIT
|
||||
if (query.limit_length)
|
||||
{
|
||||
transformStreams([&](auto & stream)
|
||||
@ -1218,17 +1218,17 @@ void InterpreterSelectQuery::executeLimit()
|
||||
size_t limit_offset = 0;
|
||||
getLimitLengthAndOffset(query, limit_length, limit_offset);
|
||||
|
||||
/// Если есть LIMIT
|
||||
/// If there is LIMIT
|
||||
if (query.limit_length)
|
||||
{
|
||||
/** Редкий случай:
|
||||
* если нет WITH TOTALS и есть подзапрос в FROM, и там на одном из уровней есть WITH TOTALS,
|
||||
* то при использовании LIMIT-а следует читать данные до конца, а не отменять выполнение запроса раньше,
|
||||
* потому что при отмене выполнения запроса, мы не получим данные для totals с удалённого сервера.
|
||||
/** Rare case:
|
||||
* if there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels,
|
||||
* then when using LIMIT, you should read the data to the end, rather than cancel the query earlier,
|
||||
* because if you cancel the query, we will not get `totals` data from the remote server.
|
||||
*
|
||||
* Ещё случай:
|
||||
* если есть WITH TOTALS и нет ORDER BY, то читать данные до конца,
|
||||
* иначе TOTALS посчитается по неполным данным.
|
||||
* Another case:
|
||||
* if there is WITH TOTALS and there is no ORDER BY, then read the data to the end,
|
||||
* otherwise TOTALS is counted according to incomplete data.
|
||||
*/
|
||||
bool always_read_till_end = false;
|
||||
|
||||
@ -1246,8 +1246,8 @@ void InterpreterSelectQuery::executeLimit()
|
||||
{
|
||||
if (subquery->group_by_with_totals)
|
||||
{
|
||||
/** NOTE Можно ещё проверять, что таблица в подзапросе - распределённая, и что она смотрит только на один шард.
|
||||
* В остальных случаях totals будет вычислен на сервере-инициаторе запроса, и читать данные до конца не обязательно.
|
||||
/** NOTE You can also check that the table in the subquery is distributed, and that it only looks at one shard.
|
||||
* In other cases, totals will be computed on the initiating server of the query, and it is not necessary to read the data to the end.
|
||||
*/
|
||||
|
||||
always_read_till_end = true;
|
||||
@ -1272,7 +1272,7 @@ void InterpreterSelectQuery::executeLimit()
|
||||
|
||||
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(SubqueriesForSets & subqueries_for_sets)
|
||||
{
|
||||
/// Если запрос не распределённый, то удалим создание временных таблиц из подзапросов (предназначавшихся для отправки на удалённые серверы).
|
||||
/// If the query is not distributed, then remove the creation of temporary tables from subqueries (intended for sending to remote servers).
|
||||
if (!(storage && storage->isRemote()))
|
||||
for (auto & elem : subqueries_for_sets)
|
||||
elem.second.table.reset();
|
||||
|
@ -29,11 +29,11 @@ void InterpreterSetQuery::executeForCurrentContext()
|
||||
|
||||
void InterpreterSetQuery::executeImpl(ASTSetQuery & ast, Context & target)
|
||||
{
|
||||
/** Значение readonly понимается следующим образом:
|
||||
* 0 - можно всё.
|
||||
* 1 - можно делать только запросы на чтение; в том числе, нельзя менять настройки.
|
||||
* 2 - можно делать только запросы на чтение и можно менять настройки, кроме настройки readonly.
|
||||
*/
|
||||
/** The `readonly` value is understood as follows:
|
||||
* 0 - everything allowed.
|
||||
* 1 - only read queries can be made; you can not change the settings.
|
||||
* 2 - You can only do read queries and you can change the settings, except for the `readonly` setting.
|
||||
*/
|
||||
|
||||
if (context.getSettingsRef().limits.readonly == 1)
|
||||
throw Exception("Cannot execute SET query in readonly mode", ErrorCodes::READONLY);
|
||||
|
@ -29,9 +29,9 @@ String InterpreterShowTablesQuery::getRewrittenQuery()
|
||||
|
||||
String database = query.from.empty() ? context.getCurrentDatabase() : query.from;
|
||||
|
||||
/** Параметр check_database_access_rights сбрасывается при обработке запроса SHOW TABLES для того,
|
||||
* чтобы все клиенты могли видеть список всех БД и таблиц в них независимо от их прав доступа
|
||||
* к этим БД.
|
||||
/** The parameter check_database_access_rights is reset when the SHOW TABLES query is processed,
|
||||
* So that all clients can see a list of all databases and tables in them regardless of their access rights
|
||||
* to these databases.
|
||||
*/
|
||||
context.assertDatabaseExists(database, false);
|
||||
|
||||
|
@ -57,7 +57,7 @@ Join::Type Join::chooseMethod(const ConstColumnPlainPtrs & key_columns, Sizes &
|
||||
keys_bytes += key_sizes[j];
|
||||
}
|
||||
|
||||
/// Если есть один числовой ключ, который помещается в 64 бита
|
||||
/// If there is one numeric key that fits in 64 bits
|
||||
if (keys_size == 1 && key_columns[0]->isNumericNotNullable())
|
||||
{
|
||||
size_t size_of_field = key_columns[0]->sizeOfField();
|
||||
@ -72,7 +72,7 @@ Join::Type Join::chooseMethod(const ConstColumnPlainPtrs & key_columns, Sizes &
|
||||
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
/// Если ключи помещаются в N бит, будем использовать хэш-таблицу по упакованным в N-бит ключам
|
||||
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys
|
||||
if (all_fixed && keys_bytes <= 16)
|
||||
return Type::keys128;
|
||||
if (all_fixed && keys_bytes <= 32)
|
||||
@ -159,7 +159,7 @@ template <> struct KeyGetterForType<Join::Type::keys256> { using Type = JoinKeyG
|
||||
template <> struct KeyGetterForType<Join::Type::hashed> { using Type = JoinKeyGetterHashed; };
|
||||
|
||||
|
||||
/// Нужно ли использовать хэш-таблицы maps_*_full, в которых запоминается, была ли строчка присоединена.
|
||||
/// Do I need to use the hash table maps_*_full, in which we remember whether the row was joined.
|
||||
static bool getFullness(ASTTableJoin::Kind kind)
|
||||
{
|
||||
return kind == ASTTableJoin::Kind::Right || kind == ASTTableJoin::Kind::Full;
|
||||
@ -259,7 +259,7 @@ void Join::setSampleBlock(const Block & block)
|
||||
|
||||
sample_block_with_columns_to_add = block;
|
||||
|
||||
/// Переносим из sample_block_with_columns_to_add ключевые столбцы в sample_block_with_keys, сохраняя порядок.
|
||||
/// Move from `sample_block_with_columns_to_add` key columns to `sample_block_with_keys`, keeping the order.
|
||||
size_t pos = 0;
|
||||
while (pos < sample_block_with_columns_to_add.columns())
|
||||
{
|
||||
@ -284,7 +284,7 @@ void Join::setSampleBlock(const Block & block)
|
||||
|
||||
namespace
|
||||
{
|
||||
/// Вставка элемента в хэш-таблицу вида ключ -> ссылка на строку, которая затем будет использоваться при JOIN-е.
|
||||
/// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN.
|
||||
template <ASTTableJoin::Strictness STRICTNESS, typename Map, typename KeyGetter>
|
||||
struct Inserter
|
||||
{
|
||||
@ -324,10 +324,10 @@ namespace
|
||||
}
|
||||
else
|
||||
{
|
||||
/** Первый элемент списка хранится в значении хэш-таблицы, остальные - в pool-е.
|
||||
* Мы будем вставлять каждый раз элемент на место второго.
|
||||
* То есть, бывший второй элемент, если он был, станет третьим, и т. п.
|
||||
*/
|
||||
/** The first element of the list is stored in the value of the hash table, the rest in the pool.
|
||||
* We will insert each time the element into the second place.
|
||||
* That is, the former second element, if it was, will be the third, and so on.
|
||||
*/
|
||||
auto elem = reinterpret_cast<typename Map::mapped_type *>(pool.alloc(sizeof(typename Map::mapped_type)));
|
||||
|
||||
elem->next = it->second.next;
|
||||
@ -431,8 +431,8 @@ bool Join::insertFromBlock(const Block & block)
|
||||
|
||||
if (getFullness(kind))
|
||||
{
|
||||
/** Переносим ключевые столбцы в начало блока.
|
||||
* Именно там их будет ожидать NonJoinedBlockInputStream.
|
||||
/** Transfer the key columns to the beginning of the block.
|
||||
* This is where NonJoinedBlockInputStream will wait for them.
|
||||
*/
|
||||
size_t key_num = 0;
|
||||
for (const auto & name : key_names_right)
|
||||
@ -446,7 +446,7 @@ bool Join::insertFromBlock(const Block & block)
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Удаляем из stored_block ключевые столбцы, так как они не нужны.
|
||||
/// Remove the key columns from stored_block, as they are not needed.
|
||||
for (const auto & name : key_names_right)
|
||||
stored_block->erase(stored_block->getPositionByName(name));
|
||||
}
|
||||
@ -642,7 +642,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
/// Rare case, when keys are constant. To avoid code bloat, simply materialize them.
|
||||
Columns materialized_columns;
|
||||
|
||||
/// Memoize key columns to work.
|
||||
/// Memoize key columns to work with.
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
key_columns[i] = block.getByName(key_names_left[i]).column.get();
|
||||
@ -661,9 +661,9 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
|
||||
size_t existing_columns = block.columns();
|
||||
|
||||
/** Если используется FULL или RIGHT JOIN, то столбцы из "левой" части надо материализовать.
|
||||
* Потому что, если они константы, то в "неприсоединённых" строчках, у них могут быть другие значения
|
||||
* - значения по-умолчанию, которые могут отличаться от значений этих констант.
|
||||
/** If you use FULL or RIGHT JOIN, then the columns from the "left" table must be materialized.
|
||||
* Because if they are constants, then in the "not joined" rows, they may have different values
|
||||
* - default values, which can differ from the values of these constants.
|
||||
*/
|
||||
if (getFullness(kind))
|
||||
{
|
||||
@ -676,7 +676,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
}
|
||||
}
|
||||
|
||||
/// Добавляем в блок новые столбцы.
|
||||
/// Add new columns to the block.
|
||||
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
|
||||
ColumnPlainPtrs added_columns(num_columns_to_add);
|
||||
|
||||
@ -691,22 +691,22 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
|
||||
size_t rows = block.rows();
|
||||
|
||||
/// Используется при ANY INNER JOIN
|
||||
/// Used with ANY INNER JOIN
|
||||
std::unique_ptr<IColumn::Filter> filter;
|
||||
|
||||
if ((kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right) && strictness == ASTTableJoin::Strictness::Any)
|
||||
filter = std::make_unique<IColumn::Filter>(rows);
|
||||
|
||||
/// Используется при ALL ... JOIN
|
||||
/// Used with ALL ... JOIN
|
||||
IColumn::Offset_t current_offset = 0;
|
||||
std::unique_ptr<IColumn::Offsets_t> offsets_to_replicate;
|
||||
|
||||
if (strictness == ASTTableJoin::Strictness::All)
|
||||
offsets_to_replicate = std::make_unique<IColumn::Offsets_t>(rows);
|
||||
|
||||
/** Для LEFT/INNER JOIN, сохранённые блоки не содержат ключи.
|
||||
* Для FULL/RIGHT JOIN, сохранённые блоки содержат ключи;
|
||||
* но они не будут использоваться на этой стадии соединения (а будут в AdderNonJoined), и их нужно пропустить.
|
||||
/** For LEFT/INNER JOIN, the saved blocks do not contain keys.
|
||||
* For FULL/RIGHT JOIN, the saved blocks contain keys;
|
||||
* but they will not be used at this stage of joining (and will be in `AdderNonJoined`), and they need to be skipped.
|
||||
*/
|
||||
size_t num_columns_to_skip = 0;
|
||||
if (getFullness(kind))
|
||||
@ -730,12 +730,12 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const
|
||||
throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT);
|
||||
}
|
||||
|
||||
/// Если ANY INNER|RIGHT JOIN - фильтруем все столбцы кроме новых.
|
||||
/// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones.
|
||||
if (filter)
|
||||
for (size_t i = 0; i < existing_columns; ++i)
|
||||
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->filter(*filter, -1);
|
||||
|
||||
/// Если ALL ... JOIN - размножаем все столбцы кроме новых.
|
||||
/// If ALL ... JOIN - we replicate all the columns except the new ones.
|
||||
if (offsets_to_replicate)
|
||||
for (size_t i = 0; i < existing_columns; ++i)
|
||||
block.safeGetByPosition(i).column = block.safeGetByPosition(i).column->replicate(*offsets_to_replicate);
|
||||
@ -746,7 +746,7 @@ void Join::joinBlockImplCross(Block & block) const
|
||||
{
|
||||
Block res = block.cloneEmpty();
|
||||
|
||||
/// Добавляем в блок новые столбцы.
|
||||
/// Add new columns to the block.
|
||||
size_t num_existing_columns = res.columns();
|
||||
size_t num_columns_to_add = sample_block_with_columns_to_add.columns();
|
||||
|
||||
@ -770,7 +770,7 @@ void Join::joinBlockImplCross(Block & block) const
|
||||
|
||||
size_t rows_left = block.rows();
|
||||
|
||||
/// NOTE Было бы оптимальнее использовать reserve, а также методы replicate для размножения значений левого блока.
|
||||
/// NOTE It would be better to use `reserve`, as well as `replicate` methods to duplicate the values of the left block.
|
||||
|
||||
for (size_t i = 0; i < rows_left; ++i)
|
||||
{
|
||||
@ -854,7 +854,7 @@ void Join::joinTotals(Block & block) const
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Будем присоединять пустые totals - из одной строчки со значениями по-умолчанию.
|
||||
/// We will attach empty `totals` - from one row with the default values.
|
||||
totals_without_keys = sample_block_with_columns_to_add.cloneEmpty();
|
||||
|
||||
for (size_t i = 0; i < totals_without_keys.columns(); ++i)
|
||||
@ -903,15 +903,15 @@ struct AdderNonJoined<ASTTableJoin::Strictness::All, Mapped>
|
||||
};
|
||||
|
||||
|
||||
/// Поток из неприсоединённых ранее строк правой таблицы.
|
||||
/// Stream from not joined earlier rows of the right table.
|
||||
class NonJoinedBlockInputStream : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
NonJoinedBlockInputStream(const Join & parent_, Block & left_sample_block, size_t max_block_size_)
|
||||
: parent(parent_), max_block_size(max_block_size_)
|
||||
{
|
||||
/** left_sample_block содержит ключи и "левые" столбцы.
|
||||
* result_sample_block - ключи, "левые" столбцы и "правые" столбцы.
|
||||
/** left_sample_block contains keys and "left" columns.
|
||||
* result_sample_block - keys, "left" columns, and "right" columns.
|
||||
*/
|
||||
|
||||
size_t num_keys = parent.key_names_left.size();
|
||||
@ -922,7 +922,7 @@ public:
|
||||
|
||||
// std::cerr << result_sample_block.dumpStructure() << "\n";
|
||||
|
||||
/// Добавляем в блок новые столбцы.
|
||||
/// Add new columns to the block.
|
||||
for (size_t i = 0; i < num_columns_right; ++i)
|
||||
{
|
||||
const ColumnWithTypeAndName & src_column = parent.sample_block_with_columns_to_add.safeGetByPosition(i);
|
||||
|
@ -117,7 +117,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
||||
auto expression_list = typeid_cast<ASTExpressionList *>(&*(function->children[0]));
|
||||
if (expression_list != nullptr)
|
||||
{
|
||||
/// Цепочка элементов выражения OR.
|
||||
/// The chain of elements of the OR expression.
|
||||
for (auto & child : expression_list->children)
|
||||
{
|
||||
auto equals = typeid_cast<ASTFunction *>(&*child);
|
||||
@ -126,7 +126,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
||||
auto equals_expression_list = typeid_cast<ASTExpressionList *>(&*(equals->children[0]));
|
||||
if ((equals_expression_list != nullptr) && (equals_expression_list->children.size() == 2))
|
||||
{
|
||||
/// Равенство expr = xN.
|
||||
/// Equality expr = xN.
|
||||
auto literal = typeid_cast<ASTLiteral *>(&*(equals_expression_list->children[1]));
|
||||
if (literal != nullptr)
|
||||
{
|
||||
@ -163,7 +163,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains()
|
||||
to_visit.push_back(Edge(to_node, &*child));
|
||||
else
|
||||
{
|
||||
/// Если узел является функцией OR, обновляем информацию про его родителей.
|
||||
/// If the node is an OR function, update the information about its parents.
|
||||
auto it = or_parent_map.find(&*child);
|
||||
if (it != or_parent_map.end())
|
||||
{
|
||||
@ -200,11 +200,11 @@ bool LogicalExpressionsOptimizer::mayOptimizeDisjunctiveEqualityChain(const Disj
|
||||
const auto & equalities = chain.second;
|
||||
const auto & equality_functions = equalities.functions;
|
||||
|
||||
/// Исключаем слишком короткие цепочки.
|
||||
/// We eliminate too short chains.
|
||||
if (equality_functions.size() < settings.optimize_min_equality_disjunction_chain_length)
|
||||
return false;
|
||||
|
||||
/// Проверяем, что правые части всех равенств имеют один и тот же тип.
|
||||
/// We check that the right-hand sides of all equalities have the same type.
|
||||
auto & first_operands = getFunctionOperands(equality_functions[0]);
|
||||
auto first_literal = static_cast<ASTLiteral *>(&*first_operands[1]);
|
||||
for (size_t i = 1; i < equality_functions.size(); ++i)
|
||||
@ -224,9 +224,9 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
||||
const auto & equalities = chain.second;
|
||||
const auto & equality_functions = equalities.functions;
|
||||
|
||||
/// 1. Создать новое выражение IN на основе информации из OR-цепочки.
|
||||
/// 1. Create a new IN expression based on information from the OR-chain.
|
||||
|
||||
/// Построить список литералов x1, ..., xN из цепочки expr = x1 OR ... OR expr = xN
|
||||
/// Construct a list of literals `x1, ..., xN` from the string `expr = x1 OR ... OR expr = xN`
|
||||
ASTPtr value_list = std::make_shared<ASTExpressionList>();
|
||||
for (const auto function : equality_functions)
|
||||
{
|
||||
@ -234,8 +234,8 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
||||
value_list->children.push_back(operands[1]);
|
||||
}
|
||||
|
||||
/// Отсортировать литералы, чтобы они были указаны в одном и том же порядке в выражении IN.
|
||||
/// Иначе они указывались бы в порядке адресов ASTLiteral, который недетерминирован.
|
||||
/// Sort the literals so that they are specified in the same order in the IN expression.
|
||||
/// Otherwise, they would be specified in the order of the ASTLiteral addresses, which is nondeterministic.
|
||||
std::sort(value_list->children.begin(), value_list->children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)
|
||||
{
|
||||
const auto val_lhs = static_cast<const ASTLiteral *>(&*lhs);
|
||||
@ -243,7 +243,7 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
||||
return val_lhs->value < val_rhs->value;
|
||||
});
|
||||
|
||||
/// Получить выражение expr из цепочки expr = x1 OR ... OR expr = xN
|
||||
/// Get the expression `expr` from the chain `expr = x1 OR ... OR expr = xN`
|
||||
ASTPtr equals_expr_lhs;
|
||||
{
|
||||
auto function = equality_functions[0];
|
||||
@ -260,14 +260,14 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
||||
expression_list->children.push_back(equals_expr_lhs);
|
||||
expression_list->children.push_back(tuple_function);
|
||||
|
||||
/// Построить выражение expr IN (x1, ..., xN)
|
||||
/// Construct the expression `expr IN (x1, ..., xN)`
|
||||
auto in_function = std::make_shared<ASTFunction>();
|
||||
in_function->name = "in";
|
||||
in_function->arguments = expression_list;
|
||||
in_function->children.push_back(in_function->arguments);
|
||||
in_function->setAlias(or_with_expression.alias);
|
||||
|
||||
/// 2. Вставить новое выражение IN.
|
||||
/// 2. Insert the new IN expression.
|
||||
|
||||
auto & operands = getFunctionOperands(or_with_expression.or_function);
|
||||
operands.push_back(in_function);
|
||||
@ -275,11 +275,11 @@ void LogicalExpressionsOptimizer::addInExpression(const DisjunctiveEqualityChain
|
||||
|
||||
void LogicalExpressionsOptimizer::cleanupOrExpressions()
|
||||
{
|
||||
/// Сохраняет для каждой оптимизированной OR-цепочки итератор на первый элемент
|
||||
/// списка операндов, которые надо удалить.
|
||||
/// Saves for each optimized OR-chain the iterator on the first element
|
||||
/// list of operands to be deleted.
|
||||
std::unordered_map<ASTFunction *, ASTs::iterator> garbage_map;
|
||||
|
||||
/// Инициализация.
|
||||
/// Initialization.
|
||||
garbage_map.reserve(processed_count);
|
||||
for (const auto & chain : disjunctive_equality_chains_map)
|
||||
{
|
||||
@ -291,7 +291,7 @@ void LogicalExpressionsOptimizer::cleanupOrExpressions()
|
||||
garbage_map.emplace(or_with_expression.or_function, operands.end());
|
||||
}
|
||||
|
||||
/// Собрать мусор.
|
||||
/// Collect garbage.
|
||||
for (const auto & chain : disjunctive_equality_chains_map)
|
||||
{
|
||||
const auto & equalities = chain.second;
|
||||
@ -314,7 +314,7 @@ void LogicalExpressionsOptimizer::cleanupOrExpressions()
|
||||
});
|
||||
}
|
||||
|
||||
/// Удалить мусор.
|
||||
/// Delete garbage.
|
||||
for (const auto & entry : garbage_map)
|
||||
{
|
||||
auto function = entry.first;
|
||||
@ -364,8 +364,8 @@ void LogicalExpressionsOptimizer::fixBrokenOrExpressions()
|
||||
parent->children.erase(first_erased, parent->children.end());
|
||||
}
|
||||
|
||||
/// Если узел OR был корнем выражения WHERE, PREWHERE или HAVING, то следует обновить этот корень.
|
||||
/// Из-за того, что имеем дело с направленным ациклическим графом, надо проверить все случаи.
|
||||
/// If the OR node was the root of the WHERE, PREWHERE, or HAVING expression, then update this root.
|
||||
/// Due to the fact that we are dealing with a directed acyclic graph, we must check all cases.
|
||||
if (select_query->where_expression && (or_function == &*(select_query->where_expression)))
|
||||
select_query->where_expression = operands[0];
|
||||
if (select_query->prewhere_expression && (or_function == &*(select_query->prewhere_expression)))
|
||||
|
@ -62,7 +62,7 @@ ProcessList::EntryPtr ProcessList::insert(
|
||||
|
||||
/// Kill query could be replaced since system.processes is continuously updated
|
||||
element->second->is_cancelled = true;
|
||||
/// В случае если запрос отменяется, данные о нем удаляются из мапа в момент отмены.
|
||||
/// If the request is canceled, the data about it is deleted from the map at the time of cancellation.
|
||||
user_process_list->second.queries.erase(element);
|
||||
}
|
||||
}
|
||||
@ -114,19 +114,19 @@ ProcessListEntry::~ProcessListEntry()
|
||||
|
||||
std::lock_guard<std::mutex> lock(parent.mutex);
|
||||
|
||||
/// Важен порядок удаления memory_tracker-ов.
|
||||
/// The order of removing memory_trackers is important.
|
||||
|
||||
String user = it->client_info.current_user;
|
||||
String query_id = it->client_info.current_query_id;
|
||||
bool is_cancelled = it->is_cancelled;
|
||||
|
||||
/// Здесь удаляется memory_tracker одного запроса.
|
||||
/// This removes the memory_tracker of one request.
|
||||
parent.cont.erase(it);
|
||||
|
||||
ProcessList::UserToQueries::iterator user_process_list = parent.user_to_queries.find(user);
|
||||
if (user_process_list != parent.user_to_queries.end())
|
||||
{
|
||||
/// В случае, если запрос отменяется, данные о нем удаляются из мапа в момент отмены, а не здесь.
|
||||
/// In case the request is canceled, the data about it is deleted from the map at the time of cancellation, and not here.
|
||||
if (!is_cancelled && !query_id.empty())
|
||||
{
|
||||
ProcessListForUser::QueryToElement::iterator element = user_process_list->second.queries.find(query_id);
|
||||
@ -134,12 +134,12 @@ ProcessListEntry::~ProcessListEntry()
|
||||
user_process_list->second.queries.erase(element);
|
||||
}
|
||||
|
||||
/// Здесь удаляется memory_tracker на пользователя. В это время, ссылающийся на него memory_tracker одного запроса не живёт.
|
||||
/// This removes the memory_tracker from the user. At this time, the memory_tracker that references it does not live.
|
||||
|
||||
/// Если запросов для пользователя больше нет, то удаляем запись.
|
||||
/// При этом также очищается MemoryTracker на пользователя, и сообщение о потреблении памяти выводится в лог.
|
||||
/// Важно иногда сбрасывать MemoryTracker, так как в нём может накапливаться смещённость
|
||||
/// в следствие того, что есть случаи, когда память может быть выделена при обработке запроса, а освобождена - позже.
|
||||
/// If there are no more queries for the user, then we delete the record.
|
||||
/// This also clears the MemoryTracker for the user, and a message about the memory consumption is output to the log.
|
||||
/// Sometimes it is important to reset the MemoryTracker, because it may accumulate skew
|
||||
/// due to the fact that there are cases when memory can be allocated while processing the request, but released later.
|
||||
if (user_process_list->second.queries.empty())
|
||||
parent.user_to_queries.erase(user_process_list);
|
||||
}
|
||||
@ -147,10 +147,10 @@ ProcessListEntry::~ProcessListEntry()
|
||||
--parent.cur_size;
|
||||
parent.have_space.signal();
|
||||
|
||||
/// Здесь удаляется memory_tracker на все запросы. В это время никакие другие memory_tracker-ы не живут.
|
||||
/// This removes memory_tracker for all requests. At this time, no other memory_trackers live.
|
||||
if (parent.cur_size == 0)
|
||||
{
|
||||
/// Сбрасываем MemoryTracker, аналогично (см. выше).
|
||||
/// Reset MemoryTracker, similarly (see above).
|
||||
parent.total_memory_tracker.logPeakMemoryUsage();
|
||||
parent.total_memory_tracker.reset();
|
||||
}
|
||||
@ -208,7 +208,7 @@ StoragePtr ProcessList::tryGetTemporaryTable(const String & query_id, const Stri
|
||||
{
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
|
||||
/// NOTE Ищем по всем user-ам. То есть, нет изоляции, и сложность O(users).
|
||||
/// NOTE We search for all user-s. That is, there is no isolation, and the complexity is O(users).
|
||||
for (const auto & user_queries : user_to_queries)
|
||||
{
|
||||
auto it = user_queries.second.queries.find(query_id);
|
||||
|
@ -80,7 +80,7 @@ static std::array<char, 16> IPv6ToBinary(const Poco::Net::IPAddress & address)
|
||||
}
|
||||
else if (Poco::Net::IPAddress::IPv4 == address.family())
|
||||
{
|
||||
/// Преобразуем в IPv6-mapped адрес.
|
||||
/// Convert to IPv6-mapped address.
|
||||
memset(res.data(), 0, 10);
|
||||
res[10] = '\xFF';
|
||||
res[11] = '\xFF';
|
||||
|
@ -100,7 +100,7 @@ void QuotaForInterval::checkAndAddReadRowsBytes(time_t current_time, const Strin
|
||||
|
||||
void QuotaForInterval::checkAndAddExecutionTime(time_t current_time, const String & quota_name, const String & user_name, Poco::Timespan amount)
|
||||
{
|
||||
/// Используется информация о внутреннем представлении Poco::Timespan.
|
||||
/// Information about internals of Poco::Timespan used.
|
||||
used.execution_time_usec += amount.totalMicroseconds();
|
||||
checkExceeded(current_time, quota_name, user_name);
|
||||
}
|
||||
|
@ -105,10 +105,10 @@ bool Set::insertFromBlock(const Block & block, bool create_ordered_set)
|
||||
data_types.reserve(keys_size);
|
||||
}
|
||||
|
||||
/// Константные столбцы справа от IN поддерживается не напрямую. Для этого, они сначала материализуется.
|
||||
/// The constant columns to the right of IN are not supported directly. For this, they first materialize.
|
||||
Columns materialized_columns;
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать
|
||||
/// Remember the columns we will work with
|
||||
for (size_t i = 0; i < keys_size; ++i)
|
||||
{
|
||||
key_columns.emplace_back(block.safeGetByPosition(i).column.get());
|
||||
@ -257,7 +257,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co
|
||||
{
|
||||
Field value = extractValueFromNode(func->arguments->children[j], *data_types[j], context);
|
||||
|
||||
/// Если хотя бы один из элементов кортежа имеет невозможное (вне диапазона типа) значение, то и весь кортеж тоже.
|
||||
/// If at least one of the elements of the tuple has an impossible (outside the range of the type) value, then the entire tuple too.
|
||||
if (value.isNull())
|
||||
break;
|
||||
|
||||
@ -329,7 +329,7 @@ ColumnPtr Set::execute(const Block & block, bool negative) const
|
||||
|
||||
const IColumn * in_column = block.safeGetByPosition(0).column.get();
|
||||
|
||||
/// Константный столбец слева от IN поддерживается не напрямую. Для этого, он сначала материализуется.
|
||||
/// The constant column to the left of IN is not supported directly. For this, it first materializes.
|
||||
ColumnPtr materialized_column = in_column->convertToFullColumnIfConst();
|
||||
if (materialized_column)
|
||||
in_column = materialized_column.get();
|
||||
@ -349,11 +349,11 @@ ColumnPtr Set::execute(const Block & block, bool negative) const
|
||||
throw Exception(message.str(), ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
/// Запоминаем столбцы, с которыми будем работать. Также проверим, что типы данных правильные.
|
||||
/// Remember the columns we will work with. Also check that the data types are correct.
|
||||
ConstColumnPlainPtrs key_columns;
|
||||
key_columns.reserve(num_key_columns);
|
||||
|
||||
/// Константные столбцы слева от IN поддерживается не напрямую. Для этого, они сначала материализуется.
|
||||
/// The constant columns to the left of IN are not supported directly. For this, they first materialize.
|
||||
Columns materialized_columns;
|
||||
|
||||
for (size_t i = 0; i < num_key_columns; ++i)
|
||||
@ -414,16 +414,16 @@ void NO_INLINE Set::executeImplCase(
|
||||
state.init(key_columns);
|
||||
size_t keys_size = key_columns.size();
|
||||
|
||||
/// NOTE Не используется оптимизация для подряд идущих одинаковых значений.
|
||||
/// NOTE Optimization is not used for consecutive identical values.
|
||||
|
||||
/// Для всех строчек
|
||||
/// For all rows
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
if (has_null_map && (*null_map)[i])
|
||||
vec_res[i] = negative;
|
||||
else
|
||||
{
|
||||
/// Строим ключ
|
||||
/// Build the key
|
||||
typename Method::Key key = state.getKey(key_columns, keys_size, i, key_sizes);
|
||||
vec_res[i] = negative ^ method.data.has(key);
|
||||
}
|
||||
@ -444,14 +444,14 @@ void NO_INLINE Set::executeArrayImpl(
|
||||
size_t keys_size = key_columns.size();
|
||||
|
||||
size_t prev_offset = 0;
|
||||
/// Для всех строчек
|
||||
/// For all rows
|
||||
for (size_t i = 0; i < rows; ++i)
|
||||
{
|
||||
UInt8 res = 0;
|
||||
/// Для всех элементов
|
||||
/// For all elements
|
||||
for (size_t j = prev_offset; j < offsets[i]; ++j)
|
||||
{
|
||||
/// Строим ключ
|
||||
/// Build the key
|
||||
typename Method::Key key = state.getKey(key_columns, keys_size, j, key_sizes);
|
||||
res |= negative ^ method.data.has(key);
|
||||
if (res)
|
||||
@ -504,9 +504,9 @@ void Set::executeArray(const ColumnArray * key_column, ColumnUInt8::Container_t
|
||||
}
|
||||
|
||||
|
||||
/// Возвращаем BoolMask.
|
||||
/// Первый элемент - может ли в диапазоне range быть элемент множества.
|
||||
/// Второй элемент - может ли в диапазоне range быть элемент не из множества.
|
||||
/// Return the BoolMask.
|
||||
/// The first element is whether the `range` element can be an element of a set.
|
||||
/// The second element is whether the element in the `range` range is not from the set.
|
||||
BoolMask Set::mayBeTrueInRange(const Range & range) const
|
||||
{
|
||||
if (!ordered_set_elements)
|
||||
@ -515,14 +515,14 @@ BoolMask Set::mayBeTrueInRange(const Range & range) const
|
||||
if (ordered_set_elements->empty())
|
||||
return {false, true};
|
||||
|
||||
/// Диапазон (-inf; +inf)
|
||||
/// Range (-inf; + inf)
|
||||
if (!range.left_bounded && !range.right_bounded)
|
||||
return {true, true};
|
||||
|
||||
const Field & left = range.left;
|
||||
const Field & right = range.right;
|
||||
|
||||
/// Диапазон (-inf; right|
|
||||
/// Range (-inf; right|
|
||||
if (!range.left_bounded)
|
||||
{
|
||||
if (range.right_included)
|
||||
@ -531,7 +531,7 @@ BoolMask Set::mayBeTrueInRange(const Range & range) const
|
||||
return {ordered_set_elements->front() < right, true};
|
||||
}
|
||||
|
||||
/// Диапазон |left; +inf)
|
||||
/// Range |left; +Inf)
|
||||
if (!range.right_bounded)
|
||||
{
|
||||
if (range.left_included)
|
||||
@ -540,7 +540,7 @@ BoolMask Set::mayBeTrueInRange(const Range & range) const
|
||||
return {ordered_set_elements->back() > left, true};
|
||||
}
|
||||
|
||||
/// Диапазон из одного значения [left].
|
||||
/// Range from one value [left].
|
||||
if (range.left_included && range.right_included && left == right)
|
||||
{
|
||||
if (std::binary_search(ordered_set_elements->begin(), ordered_set_elements->end(), left))
|
||||
@ -549,38 +549,38 @@ BoolMask Set::mayBeTrueInRange(const Range & range) const
|
||||
return {false, true};
|
||||
}
|
||||
|
||||
/// Первый элемент множества, который больше или равен left.
|
||||
/// The first element of the set that is greater than or equal to `left`.
|
||||
auto left_it = std::lower_bound(ordered_set_elements->begin(), ordered_set_elements->end(), left);
|
||||
|
||||
/// Если left не входит в диапазон (открытый диапазон), то возьмём следующий по порядку элемент множества.
|
||||
/// If `left` is not in the range (open range), then take the next element in the order of the set.
|
||||
if (!range.left_included && left_it != ordered_set_elements->end() && *left_it == left)
|
||||
++left_it;
|
||||
|
||||
/// если весь диапазон правее множества: { set } | range |
|
||||
/// if the entire range is to the right of the set: `{ set } | range |`
|
||||
if (left_it == ordered_set_elements->end())
|
||||
return {false, true};
|
||||
|
||||
/// Первый элемент множества, который строго больше right.
|
||||
/// The first element of the set, which is strictly greater than `right`.
|
||||
auto right_it = std::upper_bound(ordered_set_elements->begin(), ordered_set_elements->end(), right);
|
||||
|
||||
/// весь диапазон левее множества: | range | { set }
|
||||
/// the whole range to the left of the set: `| range | { set }`
|
||||
if (right_it == ordered_set_elements->begin())
|
||||
return {false, true};
|
||||
|
||||
/// Последний элемент множества, который меньше или равен right.
|
||||
/// The last element of the set that is less than or equal to `right`.
|
||||
--right_it;
|
||||
|
||||
/// Если right не входит в диапазон (открытый диапазон), то возьмём предыдущий по порядку элемент множества.
|
||||
/// If `right` does not enter the range (open range), then take the previous element in the order of the set.
|
||||
if (!range.right_included && *right_it == right)
|
||||
{
|
||||
/// весь диапазон левее множества, хотя открытый диапазон касается множества: | range ){ set }
|
||||
/// the entire range to the left of the set, although the open range is tangent to the set: `| range) { set }`
|
||||
if (right_it == ordered_set_elements->begin())
|
||||
return {false, true};
|
||||
|
||||
--right_it;
|
||||
}
|
||||
|
||||
/// В диапазон не попадает ни одного ключа из множества, хотя он расположен где-то посередине относительно его элементов: * * * * [ ] * * * *
|
||||
/// The range does not contain any keys from the set, although it is located somewhere in the middle relative to its elements: * * * * [ ] * * * *
|
||||
if (right_it < left_it)
|
||||
return {false, true};
|
||||
|
||||
|
@ -130,7 +130,7 @@ SetVariants::Type SetVariants::chooseMethod(const ConstColumnPlainPtrs & key_col
|
||||
return SetVariants::Type::hashed;
|
||||
}
|
||||
|
||||
/// Если есть один числовой ключ, который помещается в 64 бита
|
||||
/// If there is one numeric key that fits into 64 bits
|
||||
if (keys_size == 1 && nested_key_columns[0]->isNumericNotNullable())
|
||||
{
|
||||
size_t size_of_field = nested_key_columns[0]->sizeOfField();
|
||||
@ -145,7 +145,7 @@ SetVariants::Type SetVariants::chooseMethod(const ConstColumnPlainPtrs & key_col
|
||||
throw Exception("Logical error: numeric column has sizeOfField not in 1, 2, 4, 8.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
/// Если ключи помещаются в N бит, будем использовать хэш-таблицу по упакованным в N-бит ключам
|
||||
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys
|
||||
if (all_fixed && keys_bytes <= 16)
|
||||
return SetVariants::Type::keys128;
|
||||
if (all_fixed && keys_bytes <= 32)
|
||||
|
@ -12,7 +12,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
/// Установить настройку по имени.
|
||||
/// Set the configuration by name.
|
||||
void Settings::set(const String & name, const Field & value)
|
||||
{
|
||||
#define TRY_SET(TYPE, NAME, DEFAULT) \
|
||||
@ -26,7 +26,7 @@ void Settings::set(const String & name, const Field & value)
|
||||
#undef TRY_SET
|
||||
}
|
||||
|
||||
/// Установить настройку по имени. Прочитать сериализованное в бинарном виде значение из буфера (для межсерверного взаимодействия).
|
||||
/// Set the configuration by name. Read the binary serialized value from the buffer (for interserver interaction).
|
||||
void Settings::set(const String & name, ReadBuffer & buf)
|
||||
{
|
||||
#define TRY_SET(TYPE, NAME, DEFAULT) \
|
||||
@ -40,7 +40,7 @@ void Settings::set(const String & name, ReadBuffer & buf)
|
||||
#undef TRY_SET
|
||||
}
|
||||
|
||||
/// Пропустить сериализованное в бинарном виде значение из буфера.
|
||||
/// Skip the binary-serialized value from the buffer.
|
||||
void Settings::ignore(const String & name, ReadBuffer & buf)
|
||||
{
|
||||
#define TRY_IGNORE(TYPE, NAME, DEFAULT) \
|
||||
@ -54,7 +54,7 @@ void Settings::ignore(const String & name, ReadBuffer & buf)
|
||||
#undef TRY_IGNORE
|
||||
}
|
||||
|
||||
/** Установить настройку по имени. Прочитать значение в текстовом виде из строки (например, из конфига, или из параметра URL).
|
||||
/** Set the setting by name. Read the value in text form from a string (for example, from a config, or from a URL parameter).
|
||||
*/
|
||||
void Settings::set(const String & name, const String & value)
|
||||
{
|
||||
@ -69,8 +69,8 @@ void Settings::set(const String & name, const String & value)
|
||||
#undef TRY_SET
|
||||
}
|
||||
|
||||
/** Установить настройки из профиля (в конфиге сервера, в одном профиле может быть перечислено много настроек).
|
||||
* Профиль также может быть установлен с помощью функций set, как настройка profile.
|
||||
/** Set the settings from the profile (in the server configuration, many settings can be listed in one profile).
|
||||
* The profile can also be set using the `set` functions, like the `profile` setting.
|
||||
*/
|
||||
void Settings::setProfile(const String & profile_name, Poco::Util::AbstractConfiguration & config)
|
||||
{
|
||||
@ -84,7 +84,7 @@ void Settings::setProfile(const String & profile_name, Poco::Util::AbstractConfi
|
||||
|
||||
for (const std::string & key : config_keys)
|
||||
{
|
||||
if (key == "profile") /// Наследование одного профиля от другого.
|
||||
if (key == "profile") /// Inheritance of one profile from another.
|
||||
setProfile(config.getString(elem + "." + key), config);
|
||||
else
|
||||
set(key, config.getString(elem + "." + key));
|
||||
@ -105,8 +105,8 @@ void Settings::loadSettingsFromConfig(const String & path, const Poco::Util::Abs
|
||||
}
|
||||
}
|
||||
|
||||
/// Прочитать настройки из буфера. Они записаны как набор name-value пар, идущих подряд, заканчивающихся пустым name.
|
||||
/// Если выставлен флаг check_readonly, в настройках выставлено readonly, но пришли какие-то изменения кинуть исключение.
|
||||
/// Read the settings from the buffer. They are written as a set of name-value pairs that go successively, ending with an empty `name`.
|
||||
/// If the `check_readonly` flag is set, `readonly` is set in the preferences, but some changes have occurred - throw an exception.
|
||||
void Settings::deserialize(ReadBuffer & buf)
|
||||
{
|
||||
auto before_readonly = limits.readonly;
|
||||
@ -116,11 +116,11 @@ void Settings::deserialize(ReadBuffer & buf)
|
||||
String name;
|
||||
readBinary(name, buf);
|
||||
|
||||
/// Пустая строка - это маркер конца настроек.
|
||||
/// An empty string is the marker for the end of the settings.
|
||||
if (name.empty())
|
||||
break;
|
||||
|
||||
/// Если readonly = 2, то можно менять настройки, кроме настройки readonly.
|
||||
/// If readonly = 2, then you can change the settings, except for the readonly setting.
|
||||
if (before_readonly == 0 || (before_readonly == 2 && name != "readonly"))
|
||||
set(name, buf);
|
||||
else
|
||||
@ -128,7 +128,7 @@ void Settings::deserialize(ReadBuffer & buf)
|
||||
}
|
||||
}
|
||||
|
||||
/// Записать изменённые настройки в буфер. (Например, для отправки на удалённый сервер.)
|
||||
/// Record the changed settings to the buffer. (For example, to send to a remote server.)
|
||||
void Settings::serialize(WriteBuffer & buf) const
|
||||
{
|
||||
#define WRITE(TYPE, NAME, DEFAULT) \
|
||||
@ -142,7 +142,7 @@ void Settings::serialize(WriteBuffer & buf) const
|
||||
|
||||
limits.serialize(buf);
|
||||
|
||||
/// Пустая строка - это маркер конца настроек.
|
||||
/// An empty string is a marker for the end of the settings.
|
||||
writeStringBinary("", buf);
|
||||
|
||||
#undef WRITE
|
||||
|
@ -26,13 +26,13 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
|
||||
/** Проверка попадания Field from, имеющим тип From в диапазон значений типа To.
|
||||
* From и To - числовые типы. Могут быть типами с плавающей запятой.
|
||||
* From - это одно из UInt64, Int64, Float64,
|
||||
* тогда как To может быть также 8, 16, 32 битным.
|
||||
/** Checking for a `Field from` of `From` type falls to a range of values of type `To`.
|
||||
* `From` and `To` - numeric types. They can be floating-point types.
|
||||
* `From` is one of UInt64, Int64, Float64,
|
||||
* whereas `To` can also be 8, 16, 32 bit.
|
||||
*
|
||||
* Если попадает в диапазон, то from конвертируется в Field ближайшего к To типа.
|
||||
* Если не попадает - возвращается Field(Null).
|
||||
* If falls into a range, then `from` is converted to the `Field` closest to the `To` type.
|
||||
* If not, return Field(Null).
|
||||
*/
|
||||
|
||||
namespace
|
||||
|
@ -26,7 +26,7 @@ std::pair<Field, std::shared_ptr<IDataType>> evaluateConstantExpression(std::sha
|
||||
ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer(
|
||||
node, context, nullptr, NamesAndTypesList{{ "_dummy", std::make_shared<DataTypeUInt8>() }}).getConstActions();
|
||||
|
||||
/// В блоке должен быть хотя бы один столбец, чтобы у него было известно число строк.
|
||||
/// There must be at least one column in the block so that it knows the number of rows.
|
||||
Block block_with_constants{{ std::make_shared<ColumnConstUInt8>(1, 0), std::make_shared<DataTypeUInt8>(), "_dummy" }};
|
||||
|
||||
expr_for_constant_folding->execute(block_with_constants);
|
||||
|
@ -197,9 +197,9 @@ bool isAlreadySorted(const Block & block, const SortDescription & description)
|
||||
|
||||
PartialSortingLess less(columns_with_sort_desc);
|
||||
|
||||
/** Если строк не слишком мало, то предпримем быструю попытку проверить, что блок не сортирован.
|
||||
* Константы - наугад.
|
||||
*/
|
||||
/** If the rows are not too few, then let's make a quick attempt to verify that the block is not sorted.
|
||||
* Constants - at random.
|
||||
*/
|
||||
static constexpr size_t num_rows_to_try = 10;
|
||||
if (rows > num_rows_to_try * 5)
|
||||
{
|
||||
|
@ -20,32 +20,32 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
||||
/** Тест проверяет скорость работы хэш-таблиц, имитируя их использование для агрегации.
|
||||
* Первым аргументом указывается количество элементов, которое будет вставлено.
|
||||
* Вторым аргументом может быть указано число от 1 до 4 - номер тестируемой структуры данных.
|
||||
* Это важно, так как если запускать все тесты один за другим, то результаты будут некорректными.
|
||||
* (Из-за особенностей работы аллокатора, первый тест получает преимущество.)
|
||||
/** The test checks the speed of hash tables, simulating their use for aggregation.
|
||||
* The first argument specifies the number of elements to be inserted.
|
||||
* The second argument can be a number from 1 to 4 - the number of the data structure being tested.
|
||||
* This is important, because if you run all the tests one by one, the results will be incorrect.
|
||||
* (Due to the peculiarities of the work of the allocator, the first test takes advantage.)
|
||||
*
|
||||
* В зависимости от USE_AUTO_ARRAY, выбирается одна из структур в качестве значения.
|
||||
* USE_AUTO_ARRAY = 0 - используется std::vector (сложно-копируемая структура, sizeof = 24 байта).
|
||||
* USE_AUTO_ARRAY = 1 - используется AutoArray (структура специально разработанная для таких случаев, sizeof = 8 байт).
|
||||
* Depending on USE_AUTO_ARRAY, one of the structures is selected as the value.
|
||||
* USE_AUTO_ARRAY = 0 - uses std::vector (hard-copy structure, sizeof = 24 bytes).
|
||||
* USE_AUTO_ARRAY = 1 - uses AutoArray (a structure specially designed for such cases, sizeof = 8 bytes).
|
||||
*
|
||||
* То есть, тест также позволяет сравнить AutoArray и std::vector.
|
||||
* That is, the test also allows you to compare AutoArray and std::vector.
|
||||
*
|
||||
* Если USE_AUTO_ARRAY = 0, то HashMap уверенно обгоняет всех.
|
||||
* Если USE_AUTO_ARRAY = 1, то HashMap чуть менее серьёзно (20%) обгоняет google::dense_hash_map.
|
||||
* If USE_AUTO_ARRAY = 0, then HashMap confidently overtakes all.
|
||||
* If USE_AUTO_ARRAY = 1, then HashMap is slightly less serious (20%) ahead of google::dense_hash_map.
|
||||
*
|
||||
* При использовании HashMap, AutoArray имеет довольно серьёзное (40%) преимущество перед std::vector.
|
||||
* А при использовании других хэш-таблиц, AutoArray ещё более серьёзно обгоняет std::vector
|
||||
* (до трёх c половиной раз в случае std::unordered_map и google::sparse_hash_map).
|
||||
* When using HashMap, AutoArray has a rather serious (40%) advantage over std::vector.
|
||||
* And when using other hash tables, AutoArray even more seriously overtakes std::vector
|
||||
* (up to three and a half times in the case of std::unordered_map and google::sparse_hash_map).
|
||||
*
|
||||
* HashMap, в отличие от google::dense_hash_map, гораздо больше зависит от качества хэш-функции.
|
||||
* HashMap, unlike google::dense_hash_map, much more depends on the quality of the hash function.
|
||||
*
|
||||
* PS. Измеряйте всё сами, а то я почти запутался.
|
||||
* PS. Measure everything yourself, otherwise I'm almost confused.
|
||||
*
|
||||
* PPS. Сейчас при агрегации не используется массив агрегатных функций в качестве значений.
|
||||
* Состояния агрегатных функций были отделены от интерфейса для манипуляции с ними, и кладутся в пул.
|
||||
* Но в этом тесте осталось нечто похожее на старый сценарий использования хэш-таблиц при агрегации.
|
||||
* PPS. Now the aggregation does not use an array of aggregate functions as values.
|
||||
* States of aggregate functions were separated from the interface to manipulate them, and put in the pool.
|
||||
* But in this test, there was something similar to the old scenario of using hash tables in the aggregation.
|
||||
*/
|
||||
|
||||
#define USE_AUTO_ARRAY 0
|
||||
|
@ -40,15 +40,15 @@ struct CellWithoutZeroWithSavedHash : public HashMapCell<Key, Value, DefaultHash
|
||||
|
||||
struct Grower : public HashTableGrower<>
|
||||
{
|
||||
/// Состояние этой структуры достаточно, чтобы получить размер буфера хэш-таблицы.
|
||||
/// The state of this structure is enough to get the buffer size of the hash table.
|
||||
|
||||
/// Определяет начальный размер хэш-таблицы.
|
||||
/// Specifies the initial size of the hash table.
|
||||
static const size_t initial_size_degree = 16;
|
||||
Grower() { size_degree = initial_size_degree; }
|
||||
|
||||
// size_t max_fill = (1 << initial_size_degree) * 0.9;
|
||||
|
||||
/// Размер хэш-таблицы в ячейках.
|
||||
/// The size of the hash table in the cells.
|
||||
size_t bufSize() const { return 1 << size_degree; }
|
||||
|
||||
size_t maxFill() const { return 1 << (size_degree - 1); }
|
||||
@ -56,23 +56,23 @@ struct Grower : public HashTableGrower<>
|
||||
|
||||
size_t mask() const { return bufSize() - 1; }
|
||||
|
||||
/// Из значения хэш-функции получить номер ячейки в хэш-таблице.
|
||||
/// From the hash value, get the cell number in the hash table.
|
||||
size_t place(size_t x) const { return x & mask(); }
|
||||
|
||||
/// Следующая ячейка в цепочке разрешения коллизий.
|
||||
/// The next cell in the collision resolution chain.
|
||||
size_t next(size_t pos) const { ++pos; return pos & mask(); }
|
||||
|
||||
/// Является ли хэш-таблица достаточно заполненной. Нужно увеличить размер хэш-таблицы, или удалить из неё что-нибудь ненужное.
|
||||
/// Whether the hash table is sufficiently full. You need to increase the size of the hash table, or remove something unnecessary from it.
|
||||
bool overflow(size_t elems) const { return elems > maxFill(); }
|
||||
|
||||
/// Увеличить размер хэш-таблицы.
|
||||
/// Increase the size of the hash table.
|
||||
void increaseSize()
|
||||
{
|
||||
size_degree += size_degree >= 23 ? 1 : 2;
|
||||
// max_fill = (1 << size_degree) * 0.9;
|
||||
}
|
||||
|
||||
/// Установить размер буфера по количеству элементов хэш-таблицы. Используется при десериализации хэш-таблицы.
|
||||
/// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table.
|
||||
void set(size_t num_elems)
|
||||
{
|
||||
throw Poco::Exception(__PRETTY_FUNCTION__);
|
||||
@ -110,7 +110,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
// using Map = HashMap<Key, Value>;
|
||||
|
||||
/// Из-за WithoutZero быстрее на 0.7% (для не влезающей в L3-кэш) - 2.3% (для влезающей в L3-кэш).
|
||||
/// Due to `WithoutZero`, it's faster by 0.7% (if not fits into L3-cache) - 2.3% (if fits into L3-cache).
|
||||
using Map = HashMapTable<Key, CellWithoutZeroWithSavedHash, DefaultHash<Key>, Grower>;
|
||||
|
||||
Map map;
|
||||
|
@ -243,37 +243,37 @@ using Value = UInt64;
|
||||
|
||||
struct Grower : public HashTableGrower<>
|
||||
{
|
||||
/// Состояние этой структуры достаточно, чтобы получить размер буфера хэш-таблицы.
|
||||
/// The state of this structure is enough to get the buffer size of the hash table.
|
||||
|
||||
/// Определяет начальный размер хэш-таблицы.
|
||||
/// Defines the initial size of the hash table.
|
||||
static const size_t initial_size_degree = 16;
|
||||
Grower() { size_degree = initial_size_degree; }
|
||||
|
||||
size_t max_fill = (1 << initial_size_degree) * 0.9;
|
||||
|
||||
/// Размер хэш-таблицы в ячейках.
|
||||
/// The size of the hash table in the cells.
|
||||
size_t bufSize() const { return 1 << size_degree; }
|
||||
|
||||
size_t maxFill() const { return max_fill /*1 << (size_degree - 1)*/; }
|
||||
size_t mask() const { return bufSize() - 1; }
|
||||
|
||||
/// Из значения хэш-функции получить номер ячейки в хэш-таблице.
|
||||
/// From the hash value, get the cell number in the hash table.
|
||||
size_t place(size_t x) const { return x & mask(); }
|
||||
|
||||
/// Следующая ячейка в цепочке разрешения коллизий.
|
||||
/// The next cell in the collision resolution chain.
|
||||
size_t next(size_t pos) const { ++pos; return pos & mask(); }
|
||||
|
||||
/// Является ли хэш-таблица достаточно заполненной. Нужно увеличить размер хэш-таблицы, или удалить из неё что-нибудь ненужное.
|
||||
/// Whether the hash table is sufficiently full. You need to increase the size of the hash table, or remove something unnecessary from it.
|
||||
bool overflow(size_t elems) const { return elems > maxFill(); }
|
||||
|
||||
/// Увеличить размер хэш-таблицы.
|
||||
/// Increase the size of the hash table.
|
||||
void increaseSize()
|
||||
{
|
||||
size_degree += size_degree >= 23 ? 1 : 2;
|
||||
max_fill = (1 << size_degree) * 0.9;
|
||||
}
|
||||
|
||||
/// Установить размер буфера по количеству элементов хэш-таблицы. Используется при десериализации хэш-таблицы.
|
||||
/// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table.
|
||||
void set(size_t num_elems)
|
||||
{
|
||||
throw Poco::Exception(__PRETTY_FUNCTION__);
|
||||
@ -317,7 +317,7 @@ int main(int argc, char ** argv)
|
||||
|
||||
//using Map = HashMap<Key, Value>;
|
||||
|
||||
/// Сохранение хэша ускоряет ресайзы примерно в 2 раза, и общую производительность - на 6-8%.
|
||||
/// Saving the hash accelerates the resize by about 2 times, and the overall performance by 6-8%.
|
||||
using Map = HashMapWithSavedHash<Key, Value, DefaultHash<Key>, Grower>;
|
||||
|
||||
Map map;
|
||||
|
@ -20,7 +20,7 @@
|
||||
#endif
|
||||
|
||||
|
||||
/** Выполнять так:
|
||||
/** Do this:
|
||||
for file in MobilePhoneModel PageCharset Params URLDomain UTMSource Referer URL Title; do
|
||||
for size in 30000 100000 300000 1000000 5000000; do
|
||||
echo
|
||||
@ -207,7 +207,7 @@ inline bool compare_byIntSSE(const char * p1, const char * p2)
|
||||
|
||||
inline bool compare_byFloatSSE(const char * p1, const char * p2)
|
||||
{
|
||||
return !_mm_movemask_ps(_mm_cmpneq_ps( /// Кажется, некорректно при сравнении субнормальных float-ов.
|
||||
return !_mm_movemask_ps(_mm_cmpneq_ps( /// Looks like incorrect while comparing subnormal floats.
|
||||
_mm_loadu_ps(reinterpret_cast<const float *>(p1)),
|
||||
_mm_loadu_ps(reinterpret_cast<const float *>(p2))));
|
||||
}
|
||||
|
@ -23,7 +23,7 @@
|
||||
#endif
|
||||
|
||||
|
||||
/** Выполнять так:
|
||||
/** Do this:
|
||||
for file in MobilePhoneModel PageCharset Params URLDomain UTMSource Referer URL Title; do
|
||||
for size in 30000 100000 300000 1000000 5000000; do
|
||||
echo
|
||||
|
@ -26,7 +26,7 @@ namespace DB
|
||||
}
|
||||
|
||||
|
||||
/// Упрощённый вариант класса StorageDistributed.
|
||||
/// Simplified version of the StorageDistributed class.
|
||||
class StorageDistributedFake : private ext::shared_ptr_helper<StorageDistributedFake>, public DB::IStorage
|
||||
{
|
||||
friend class ext::shared_ptr_helper<StorageDistributedFake>;
|
||||
@ -107,7 +107,7 @@ void reorder(DB::IAST * ast);
|
||||
|
||||
TestEntries entries =
|
||||
{
|
||||
/// Тривиальный запрос.
|
||||
/// Trivial query.
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -217,7 +217,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция IN / глубина 1
|
||||
/// Section IN / depth 1
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -291,7 +291,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция NOT IN / глубина 1
|
||||
/// Section NOT IN / depth 1
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -347,7 +347,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция GLOBAL IN / глубина 1
|
||||
/// Section GLOBAL IN / depth 1
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -385,7 +385,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция GLOBAL NOT IN / глубина 1
|
||||
/// Section GLOBAL NOT IN / depth 1
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -423,7 +423,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция JOIN / глубина 1
|
||||
/// Section JOIN / depth 1
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -479,7 +479,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция GLOBAL JOIN / глубина 1
|
||||
/// Section GLOBAL JOIN / depth 1
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -517,7 +517,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция JOIN / глубина 1 / 2 подзапроса.
|
||||
/// Section JOIN / depth 1 / 2 of the subquery.
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -564,7 +564,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция IN / глубина 1 / таблица на уровне 2
|
||||
/// Section IN / depth 1 / table at level 2
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -620,7 +620,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция GLOBAL IN / глубина 1 / таблица на уровне 2
|
||||
/// Section GLOBAL IN / depth 1 / table at level 2
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -658,7 +658,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция IN на уровне 1, секция GLOBAL IN на уровне 2.
|
||||
/// Section IN at level 1, GLOBAL IN section at level 2.
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -687,7 +687,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция JOIN / глубина 1 / таблица на уровне 2
|
||||
/// Section JOIN / depth 1 / table at level 2
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -743,7 +743,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция IN / глубина 2
|
||||
/// Section IN / depth 2
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -817,7 +817,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция JOIN / глубина 2
|
||||
/// Section JOIN / depth 2
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -864,7 +864,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция JOIN / глубина 2
|
||||
/// Section JOIN / depth 2
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -920,7 +920,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция JOIN / секция IN
|
||||
/// Section JOIN / section IN
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -967,7 +967,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Табличная функция.
|
||||
/// Table function.
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -1032,7 +1032,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Секция IN / глубина 2 / две распределённые таблицы
|
||||
/// Section IN / depth 2 / two distributed tables
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -1043,7 +1043,7 @@ TestEntries entries =
|
||||
true
|
||||
},
|
||||
|
||||
/// Агрегатная функция.
|
||||
/// Aggregate function.
|
||||
|
||||
{
|
||||
__LINE__,
|
||||
@ -1187,7 +1187,7 @@ TestResult check(const TestEntry & entry)
|
||||
auto & settings = context.getSettingsRef();
|
||||
settings.distributed_product_mode = entry.mode;
|
||||
|
||||
/// Парсить и обработать входящий запрос.
|
||||
/// Parse and process the incoming query.
|
||||
DB::ASTPtr ast_input;
|
||||
if (!parse(ast_input, entry.input))
|
||||
return TestResult(false, "parse error");
|
||||
@ -1215,12 +1215,12 @@ TestResult check(const TestEntry & entry)
|
||||
if (success != entry.expected_success)
|
||||
return TestResult(false, "unexpected result");
|
||||
|
||||
/// Парсить ожидаемый результат.
|
||||
/// Parse the expected result.
|
||||
DB::ASTPtr ast_expected;
|
||||
if (!parse(ast_expected, entry.expected_output))
|
||||
return TestResult(false, "parse error");
|
||||
|
||||
/// Сравнить обработанный запрос и ожидаемый результат.
|
||||
/// Compare the processed query and the expected result.
|
||||
bool res = equals(ast_input, ast_expected);
|
||||
std::string output = DB::queryToString(ast_input);
|
||||
|
||||
|
@ -34,7 +34,7 @@ void reorder(DB::IAST * ast);
|
||||
|
||||
void run()
|
||||
{
|
||||
/// NOTE: Запросы не всегда реалистичные, однако лишь синтаксис нас интересует.
|
||||
/// NOTE: Queries are not always realistic, but we are only interested in the syntax.
|
||||
TestEntries entries =
|
||||
{
|
||||
{
|
||||
@ -204,7 +204,7 @@ TestResult check(const TestEntry & entry)
|
||||
{
|
||||
try
|
||||
{
|
||||
/// Парсить и оптимизировать входящий запрос.
|
||||
/// Parse and optimize the incoming query.
|
||||
DB::ASTPtr ast_input;
|
||||
if (!parse(ast_input, entry.input))
|
||||
return TestResult(false, "parse error");
|
||||
@ -217,12 +217,12 @@ TestResult check(const TestEntry & entry)
|
||||
DB::LogicalExpressionsOptimizer optimizer(select_query, settings);
|
||||
optimizer.perform();
|
||||
|
||||
/// Парсить ожидаемый результат.
|
||||
/// Parse the expected result.
|
||||
DB::ASTPtr ast_expected;
|
||||
if (!parse(ast_expected, entry.expected_output))
|
||||
return TestResult(false, "parse error");
|
||||
|
||||
/// Сравнить оптимизированный запрос и ожидаемый результат.
|
||||
/// Compare the optimized query and the expected result.
|
||||
bool res = equals(ast_input, ast_expected);
|
||||
std::string output = DB::queryToString(ast_input);
|
||||
|
||||
|
@ -25,7 +25,7 @@ try
|
||||
Logger::root().setChannel(channel);
|
||||
Logger::root().setLevel("trace");
|
||||
|
||||
/// Заранее инициализируем DateLUT, чтобы первая инициализация потом не влияла на измеряемую скорость выполнения.
|
||||
/// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed.
|
||||
DateLUT::instance();
|
||||
|
||||
Context context;
|
||||
|
@ -35,7 +35,7 @@ struct TestDescriptor
|
||||
|
||||
using TestSet = std::vector<TestDescriptor>;
|
||||
|
||||
/// Описание тестов.
|
||||
/// Tests description.
|
||||
|
||||
TestSet test_set =
|
||||
{
|
||||
|
@ -48,7 +48,7 @@ ASTAlterQuery::ASTAlterQuery(StringRange range_) : IAST(range_)
|
||||
{
|
||||
}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
/** Get the text that identifies this element. */
|
||||
String ASTAlterQuery::getID() const
|
||||
{
|
||||
return ("AlterQuery_" + database + "_" + table);
|
||||
|
@ -38,7 +38,7 @@ String ASTFunction::getColumnName() const
|
||||
return res;
|
||||
}
|
||||
|
||||
/** Получить текст, который идентифицирует этот элемент. */
|
||||
/** Get the text that identifies this element. */
|
||||
String ASTFunction::getID() const
|
||||
{
|
||||
return "Function_" + name;
|
||||
@ -102,10 +102,10 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
||||
{
|
||||
settings.ostr << (settings.hilite ? hilite_operator : "") << func[1] << (settings.hilite ? hilite_none : "");
|
||||
|
||||
/** Особо дурацкий случай. Если у нас унарный минус перед литералом, являющимся отрицательным числом:
|
||||
* "-(-1)" или "- -1", то это нельзя форматировать как --1, так как это будет воспринято как комментарий.
|
||||
* Вместо этого, добавим пробел.
|
||||
* PS. Нельзя просто попросить добавить скобки - см. formatImpl для ASTLiteral.
|
||||
/** A particularly stupid case. If we have a unary minus before a literal that is a negative number
|
||||
* "-(-1)" or "- -1", this can not be formatted as `--1`, since this will be interpreted as a comment.
|
||||
* Instead, add a space.
|
||||
* PS. You can not just ask to add parentheses - see formatImpl for ASTLiteral.
|
||||
*/
|
||||
if (name == "negate" && typeid_cast<const ASTLiteral *>(&*arguments->children[0]))
|
||||
settings.ostr << ' ';
|
||||
@ -116,9 +116,9 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
||||
}
|
||||
}
|
||||
|
||||
/** need_parens - нужны ли скобки вокруг выражения с оператором.
|
||||
* Они нужны, только если это выражение входит в другое выражение с оператором.
|
||||
*/
|
||||
/** need_parens - do I need parentheses around the expression with the operator.
|
||||
* They are needed only if this expression is included in another expression with the operator.
|
||||
*/
|
||||
|
||||
if (!written && arguments->children.size() == 2)
|
||||
{
|
||||
|
@ -19,7 +19,7 @@ void ASTIdentifier::formatImplWithoutAlias(const FormatSettings & settings, Form
|
||||
settings.ostr << (settings.hilite ? hilite_none : "");
|
||||
};
|
||||
|
||||
/// Простой или составной идентификатор?
|
||||
/// A simple or compound identifier?
|
||||
|
||||
if (children.size() > 1)
|
||||
{
|
||||
|
@ -58,8 +58,8 @@ void ASTSelectQuery::renameColumns(const ASTSelectQuery & source)
|
||||
|
||||
for (size_t i = 0; i < from.size(); ++i)
|
||||
{
|
||||
/// Если столбец имеет алиас, то он должен совпадать с названием исходного столбца.
|
||||
/// В противном случае мы ему присваиваем алиас, если требуется.
|
||||
/// If the column has an alias, it must match the name of the original column.
|
||||
/// Otherwise, we assign it an alias, if required.
|
||||
if (!to[i]->tryGetAlias().empty())
|
||||
{
|
||||
if (to[i]->tryGetAlias() != from[i]->getAliasOrColumnName())
|
||||
@ -76,9 +76,9 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n
|
||||
ASTPtr result = std::make_shared<ASTExpressionList>();
|
||||
ASTs asts = select_expression_list->children;
|
||||
|
||||
/// Создать отображение.
|
||||
/// Create a mapping.
|
||||
|
||||
/// Элемент отображения.
|
||||
/// The element of mapping.
|
||||
struct Arrow
|
||||
{
|
||||
Arrow() = default;
|
||||
@ -90,15 +90,15 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n
|
||||
bool is_selected = false;
|
||||
};
|
||||
|
||||
/// Отображение одного SELECT выражения в другое.
|
||||
/// Mapping of one SELECT expression to another.
|
||||
using Mapping = std::vector<Arrow>;
|
||||
|
||||
Mapping mapping(asts.size());
|
||||
|
||||
/// На какой позиции в SELECT-выражении находится соответствующий столбец из column_names.
|
||||
/// On which position in the SELECT expression is the corresponding column from `column_names`.
|
||||
std::vector<size_t> positions_of_required_columns(required_column_names.size());
|
||||
|
||||
/// Не будем выбрасывать выражения, содержащие функцию arrayJoin.
|
||||
/// We will not throw out expressions that contain the `arrayJoin` function.
|
||||
for (size_t i = 0; i < asts.size(); ++i)
|
||||
{
|
||||
if (hasArrayJoin(asts[i]))
|
||||
@ -129,7 +129,7 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n
|
||||
mapping[positions_of_required_columns_in_subquery_order[i]] = Arrow(positions_of_required_columns[i]);
|
||||
|
||||
|
||||
/// Составить новое выражение.
|
||||
/// Construct a new expression.
|
||||
for (const auto & arrow : mapping)
|
||||
{
|
||||
if (arrow.is_selected)
|
||||
@ -146,9 +146,9 @@ void ASTSelectQuery::rewriteSelectExpressionList(const Names & required_column_n
|
||||
}
|
||||
select_expression_list = result;
|
||||
|
||||
/** NOTE: Может показаться, что мы могли испортить запрос, выбросив выражение с алиасом, который используется где-то еще.
|
||||
* Такого произойти не может, потому что этот метод вызывается всегда для запроса, на котором хоть раз создавали
|
||||
* ExpressionAnalyzer, что гарантирует, что в нем все алиасы уже подставлены. Не совсем очевидная логика.
|
||||
/** NOTE: It might seem that we could spoil the query by throwing an expression with an alias that is used somewhere else.
|
||||
* This can not happen, because this method is always called for a query, for which ExpressionAnalyzer was created at least once,
|
||||
* which ensures that all aliases in it are already set. Not quite obvious logic.
|
||||
*/
|
||||
}
|
||||
|
||||
@ -156,7 +156,7 @@ ASTPtr ASTSelectQuery::clone() const
|
||||
{
|
||||
auto ptr = cloneImpl(true);
|
||||
|
||||
/// Установить указатели на предыдущие запросы SELECT.
|
||||
/// Set pointers to previous SELECT queries.
|
||||
ASTPtr current = ptr;
|
||||
static_cast<ASTSelectQuery *>(current.get())->prev_union_all = nullptr;
|
||||
ASTPtr next = static_cast<ASTSelectQuery *>(current.get())->next_union_all;
|
||||
@ -187,15 +187,15 @@ std::shared_ptr<ASTSelectQuery> ASTSelectQuery::cloneImpl(bool traverse_union_al
|
||||
|
||||
#define CLONE(member) if (member) { res->member = member->clone(); res->children.push_back(res->member); }
|
||||
|
||||
/** NOTE Члены должны клонироваться точно в таком же порядке,
|
||||
* в каком они были вставлены в children в ParserSelectQuery.
|
||||
* Это важно, потому что из имён children-ов составляется идентификатор (getTreeID),
|
||||
* который может быть использован для идентификаторов столбцов в случае подзапросов в операторе IN.
|
||||
* При распределённой обработке запроса, в случае, если один из серверов localhost, а другой - нет,
|
||||
* запрос на localhost выполняется в рамках процесса и при этом клонируется,
|
||||
* а на удалённый сервер запрос отправляется в текстовом виде по TCP.
|
||||
* И если порядок при клонировании не совпадает с порядком при парсинге,
|
||||
* то на разных серверах получатся разные идентификаторы.
|
||||
/** NOTE Members must clone exactly in the same order,
|
||||
* in which they were inserted into `children` in ParserSelectQuery.
|
||||
* This is important because of the children's names the identifier (getTreeID) is compiled,
|
||||
* which can be used for column identifiers in the case of subqueries in the IN statement.
|
||||
* For distributed query processing, in case one of the servers is localhost and the other one is not,
|
||||
* localhost query is executed within the process and is cloned,
|
||||
* and the request is sent to the remote server in text form via TCP.
|
||||
* And if the cloning order does not match the parsing order,
|
||||
* then different servers will get different identifiers.
|
||||
*/
|
||||
CLONE(select_expression_list)
|
||||
CLONE(tables)
|
||||
@ -320,8 +320,8 @@ void ASTSelectQuery::formatQueryImpl(const FormatSettings & s, FormatState & sta
|
||||
{
|
||||
s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << "UNION ALL " << s.nl_or_ws << (s.hilite ? hilite_none : "");
|
||||
|
||||
// NOTE Мы можем безопасно применить static_cast вместо typeid_cast, потому что знаем, что в цепочке UNION ALL
|
||||
// имеются только деревья типа SELECT.
|
||||
// NOTE We can safely apply `static_cast` instead of `typeid_cast` because we know that in the `UNION ALL` chain
|
||||
// there are only trees of type SELECT.
|
||||
const ASTSelectQuery & next_ast = static_cast<const ASTSelectQuery &>(*next_union_all);
|
||||
|
||||
next_ast.formatImpl(s, state, frame);
|
||||
|
@ -10,7 +10,7 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta
|
||||
{
|
||||
if (!alias.empty())
|
||||
{
|
||||
/// Если мы уже ранее вывели этот узел в другом месте запроса, то теперь достаточно вывести лишь алиас.
|
||||
/// If we have previously output this node elsewhere in the query, now it is enough to output only the alias.
|
||||
if (!state.printed_asts_with_alias.emplace(frame.current_select, alias).second)
|
||||
{
|
||||
WriteBufferFromOStream wb(settings.ostr, 32);
|
||||
@ -19,7 +19,7 @@ void ASTWithAlias::formatImpl(const FormatSettings & settings, FormatState & sta
|
||||
}
|
||||
}
|
||||
|
||||
/// Если есть алиас, то требуются скобки вокруг всего выражения, включая алиас. Потому что запись вида 0 AS x + 0 синтаксически некорректна.
|
||||
/// If there is an alias, then parentheses are required around the entire expression, including the alias. Because a record of the form `0 AS x + 0` is syntactically invalid.
|
||||
if (frame.need_parens && !alias.empty())
|
||||
settings.ostr <<'(';
|
||||
|
||||
|
@ -84,7 +84,7 @@ bool ParserParenthesisExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, P
|
||||
|
||||
ASTExpressionList & expr_list = typeid_cast<ASTExpressionList &>(*contents_node);
|
||||
|
||||
/// пустое выражение в скобках недопустимо
|
||||
/// empty expression in parentheses is not allowed
|
||||
if (expr_list.children.empty())
|
||||
{
|
||||
expected = "non-empty parenthesized list of expressions";
|
||||
@ -137,14 +137,14 @@ bool ParserIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pa
|
||||
{
|
||||
Pos begin = pos;
|
||||
|
||||
/// Идентификатор в обратных кавычках
|
||||
/// Identifier in backquotes
|
||||
if (pos != end && *pos == '`')
|
||||
{
|
||||
ReadBufferFromMemory buf(pos, end - pos);
|
||||
String s;
|
||||
readBackQuotedString(s, buf);
|
||||
|
||||
if (s.empty()) /// Не разрешены идентификаторы "пустая строка".
|
||||
if (s.empty()) /// Identifiers "empty string" are not allowed.
|
||||
return false;
|
||||
|
||||
pos += buf.count();
|
||||
@ -190,7 +190,7 @@ bool ParserCompoundIdentifier::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos
|
||||
|
||||
node = std::make_shared<ASTIdentifier>(StringRange(begin, pos), name);
|
||||
|
||||
/// В children запомним идентификаторы-составляющие, если их больше одного.
|
||||
/// In `children`, remember the identifiers-components, if there are more than one.
|
||||
if (list.children.size() > 1)
|
||||
node->children.insert(node->children.end(), list.children.begin(), list.children.end());
|
||||
|
||||
@ -239,10 +239,10 @@ bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pars
|
||||
if (!close.ignore(pos, end, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
/** Проверка на распространённый случай ошибки - часто из-за сложности квотирования аргументов командной строки,
|
||||
* в запрос попадает выражение вида toDate(2014-01-01) вместо toDate('2014-01-01').
|
||||
* Если не сообщить, что первый вариант - ошибка, то аргумент будет проинтерпретирован как 2014 - 01 - 01 - некоторое число,
|
||||
* и запрос тихо вернёт неожиданный результат.
|
||||
/** Check for a common error case - often due to the complexity of quoting command-line arguments,
|
||||
* an expression of the form toDate(2014-01-01) appears in the query instead of toDate('2014-01-01').
|
||||
* If you do not report that the first option is an error, then the argument will be interpreted as 2014 - 01 - 01 - some number,
|
||||
* and the query silently returns an unexpected result.
|
||||
*/
|
||||
if (typeid_cast<const ASTIdentifier &>(*identifier).name == "toDate"
|
||||
&& contents_end - contents_begin == strlen("2014-01-01")
|
||||
@ -262,7 +262,7 @@ bool ParserFunction::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_pars
|
||||
, ErrorCodes::SYNTAX_ERROR);
|
||||
}
|
||||
|
||||
/// У параметрической агрегатной функции - два списка (параметры и аргументы) в круглых скобках. Пример: quantile(0.9)(x).
|
||||
/// The parametric aggregate function has two lists (parameters and arguments) in parentheses. Example: quantile(0.9)(x).
|
||||
if (open.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
/// Parametric aggregate functions cannot have DISTINCT in parameters list.
|
||||
@ -664,9 +664,9 @@ bool ParserAliasImpl<ParserIdentifier>::parseImpl(Pos & pos, Pos end, ASTPtr & n
|
||||
|
||||
if (!has_as_word)
|
||||
{
|
||||
/** В этом случае алиас не может совпадать с ключевым словом - для того,
|
||||
* чтобы в запросе "SELECT x FROM t", слово FROM не считалось алиасом,
|
||||
* а в запросе "SELECT x FRO FROM t", слово FRO считалось алиасом.
|
||||
/** In this case, the alias can not match the keyword -
|
||||
* so that in the query "SELECT x FROM t", the word FROM was not considered an alias,
|
||||
* and in the query "SELECT x FRO FROM t", the word FRO was considered an alias.
|
||||
*/
|
||||
|
||||
const String & name = static_cast<const ASTIdentifier &>(*node.get()).name;
|
||||
@ -772,24 +772,24 @@ bool ParserWithOptionalAliasImpl<ParserAlias>::parseImpl(Pos & pos, Pos end, AST
|
||||
if (!elem_parser->parse(pos, end, node, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
/** Маленький хак.
|
||||
/** Little hack.
|
||||
*
|
||||
* В секции SELECT мы разрешаем парсить алиасы без указания ключевого слова AS.
|
||||
* Эти алиасы не могут совпадать с ключевыми словами запроса.
|
||||
* А само выражение может быть идентификатором, совпадающем с ключевым словом.
|
||||
* Например, столбец может называться where. И в запросе может быть написано SELECT where AS x FROM table или даже SELECT where x FROM table.
|
||||
* Даже может быть написано SELECT where AS from FROM table, но не может быть написано SELECT where from FROM table.
|
||||
* Смотрите подробнее в реализации ParserAlias.
|
||||
* In the SELECT section, we allow parsing aliases without specifying the AS keyword.
|
||||
* These aliases can not be the same as the query keywords.
|
||||
* And the expression itself can be an identifier that matches the keyword.
|
||||
* For example, a column may be called where. And in the query it can be written `SELECT where AS x FROM table` or even `SELECT where x FROM table`.
|
||||
* Even can be written `SELECT where AS from FROM table`, but it can not be written `SELECT where from FROM table`.
|
||||
* See the ParserAlias implementation for details.
|
||||
*
|
||||
* Но возникает небольшая проблема - неудобное сообщение об ошибке, если в секции SELECT в конце есть лишняя запятая.
|
||||
* Хотя такая ошибка очень распространена. Пример: SELECT x, y, z, FROM tbl
|
||||
* Если ничего не предпринять, то это парсится как выбор столбца с именем FROM и алиасом tbl.
|
||||
* Чтобы избежать такой ситуации, мы не разрешаем парсить алиас без ключевого слова AS для идентификатора с именем FROM.
|
||||
* But there is a small problem - an inconvenient error message if there is an extra comma in the SELECT section at the end.
|
||||
* Although this error is very common. Example: `SELECT x, y, z, FROM tbl`
|
||||
* If you do nothing, it's parsed as a column with the name FROM and alias tbl.
|
||||
* To avoid this situation, we do not allow the parsing of the alias without the AS keyword for the identifier with the name FROM.
|
||||
*
|
||||
* Замечание: это также фильтрует случай, когда идентификатор квотирован.
|
||||
* Пример: SELECT x, y, z, `FROM` tbl. Но такой случай можно было бы разрешить.
|
||||
* Note: this also filters the case when the identifier is quoted.
|
||||
* Example: SELECT x, y, z, `FROM` tbl. But such a case could be solved.
|
||||
*
|
||||
* В дальнейшем было бы проще запретить неквотированные идентификаторы, совпадающие с ключевыми словами.
|
||||
* In the future it would be easier to disallow unquoted identifiers that match the keywords.
|
||||
*/
|
||||
bool allow_alias_without_as_keyword_now = allow_alias_without_as_keyword;
|
||||
if (allow_alias_without_as_keyword)
|
||||
|
@ -140,7 +140,7 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// пробуем найти какой-нибудь из допустимых операторов
|
||||
/// try to find any of the valid operators
|
||||
|
||||
const char ** it;
|
||||
for (it = operators; *it; it += 2)
|
||||
@ -155,17 +155,17 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// функция, соответствующая оператору
|
||||
/// the function corresponding to the operator
|
||||
auto function = std::make_shared<ASTFunction>();
|
||||
|
||||
/// аргументы функции
|
||||
/// function arguments
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
|
||||
ASTPtr elem;
|
||||
if (!(remaining_elem_parser ? remaining_elem_parser : first_elem_parser)->parse(pos, end, elem, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
/// первым аргументом функции будет предыдущий элемент, вторым - следующий
|
||||
/// the first argument of the function is the previous element, the second is the next one
|
||||
function->range.first = begin;
|
||||
function->range.second = pos;
|
||||
function->name = it[1];
|
||||
@ -177,8 +177,8 @@ bool ParserLeftAssociativeBinaryOperatorList::parseImpl(Pos & pos, Pos end, ASTP
|
||||
exp_list->range.first = begin;
|
||||
exp_list->range.second = pos;
|
||||
|
||||
/** специальное исключение для оператора доступа к элементу массива x[y], который
|
||||
* содержит инфиксную часть '[' и суффиксную ']' (задаётся в виде '[')
|
||||
/** special exception for the access operator to the element of the array `x[y]`, which
|
||||
* contains the infix part '[' and the suffix ''] '(specified as' [')
|
||||
*/
|
||||
if (0 == strcmp(it[0], "["))
|
||||
{
|
||||
@ -236,8 +236,8 @@ bool ParserVariableArityOperatorList::parseImpl(Pos & pos, Pos end, ASTPtr & nod
|
||||
|
||||
bool ParserBetweenExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
/// Для выражения (subject BETWEEN left AND right)
|
||||
/// создаём AST такое же, как для (subject >= left AND subject <= right).
|
||||
/// For the expression (subject BETWEEN left AND right)
|
||||
/// create an AST the same as for (subject> = left AND subject <= right).
|
||||
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
ParserString s_between("BETWEEN", true, true);
|
||||
@ -273,7 +273,7 @@ bool ParserBetweenExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
|
||||
if (!elem_parser.parse(pos, end, right, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
/// функция AND
|
||||
/// AND function
|
||||
auto f_and = std::make_shared<ASTFunction>();
|
||||
auto args_and = std::make_shared<ASTExpressionList>();
|
||||
|
||||
@ -354,10 +354,10 @@ bool ParserTernaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr & nod
|
||||
if (!elem_parser.parse(pos, end, elem_else, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
/// функция, соответствующая оператору
|
||||
/// the function corresponding to the operator
|
||||
auto function = std::make_shared<ASTFunction>();
|
||||
|
||||
/// аргументы функции
|
||||
/// function arguments
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
|
||||
function->range.first = begin;
|
||||
@ -450,7 +450,7 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
|
||||
{
|
||||
ParserWhiteSpaceOrComments ws;
|
||||
|
||||
/// пробуем найти какой-нибудь из допустимых операторов
|
||||
/// try to find any of the valid operators
|
||||
Pos begin = pos;
|
||||
const char ** it;
|
||||
for (it = operators; *it; it += 2)
|
||||
@ -462,13 +462,13 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// Позволяем парсить цепочки вида NOT NOT x. Это хак.
|
||||
/** Так сделано, потому что среди унарных операторов есть только минус и NOT.
|
||||
* Но для минуса цепочку из унарных операторов не требуется поддерживать.
|
||||
/// Let's parse chains of the form `NOT NOT x`. This is hack.
|
||||
/** This is done, because among the unary operators there is only a minus and NOT.
|
||||
* But for a minus the chain of unary operators does not need to be supported.
|
||||
*/
|
||||
if (it[0] && 0 == strncmp(it[0], "NOT", 3))
|
||||
{
|
||||
/// Было ли чётное количество NOT.
|
||||
/// Was there an even number of NOTs.
|
||||
bool even = false;
|
||||
|
||||
const char ** jt;
|
||||
@ -490,7 +490,7 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
|
||||
}
|
||||
|
||||
if (even)
|
||||
it = jt; /// Зануляем результат парсинга первого NOT. Получается, как будто цепочки NOT нет вообще.
|
||||
it = jt; /// Zero the result of parsing the first NOT. It turns out, as if there is no `NOT` chain at all.
|
||||
}
|
||||
|
||||
ASTPtr elem;
|
||||
@ -501,10 +501,10 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
|
||||
node = elem;
|
||||
else
|
||||
{
|
||||
/// функция, соответствующая оператору
|
||||
/// the function corresponding to the operator
|
||||
auto function = std::make_shared<ASTFunction>();
|
||||
|
||||
/// аргументы функции
|
||||
/// function arguments
|
||||
auto exp_list = std::make_shared<ASTExpressionList>();
|
||||
|
||||
function->range.first = begin;
|
||||
@ -526,7 +526,7 @@ bool ParserPrefixUnaryOperatorExpression::parseImpl(Pos & pos, Pos end, ASTPtr &
|
||||
|
||||
bool ParserUnaryMinusExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
/// В качестве исключения, отрицательные числа должны парситься, как литералы, а не как применение оператора.
|
||||
/// As an exception, negative numbers should be parsed as literals, and not as an application of the operator.
|
||||
|
||||
if (pos < end && *pos == '-')
|
||||
{
|
||||
|
@ -24,7 +24,7 @@ const char * IAST::hilite_alias = "\033[0;32m";
|
||||
const char * IAST::hilite_none = "\033[0m";
|
||||
|
||||
|
||||
/// Квотировать идентификатор обратными кавычками, если это требуется.
|
||||
/// Quota the identifier with backquotes, if required.
|
||||
String backQuoteIfNeed(const String & x)
|
||||
{
|
||||
String res(x.size(), '\0');
|
||||
|
@ -23,7 +23,7 @@ bool ParserNestedTable::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
|
||||
Pos begin = pos;
|
||||
|
||||
/// Пока name == 'Nested', возможно потом появятся альтернативные вложенные структуры данных
|
||||
/// For now `name == 'Nested'`, probably alternative nested data structures will appear
|
||||
if (!name_p.parse(pos, end, name, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
@ -244,7 +244,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
/// Список столбцов
|
||||
/// Columns list
|
||||
if (s_lparen.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
@ -263,7 +263,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
if (!engine_p.parse(pos, end, storage, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
/// Для engine VIEW необходимо так же считать запрос AS SELECT
|
||||
/// For engine VIEW, you also need to read AS SELECT
|
||||
if (storage && (typeid_cast<ASTFunction &>(*storage).name == "View"
|
||||
|| typeid_cast<ASTFunction &>(*storage).name == "MaterializedView"))
|
||||
{
|
||||
@ -314,7 +314,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// Опционально - может быть указана ENGINE.
|
||||
/// Optional - ENGINE can be specified.
|
||||
engine_p.parse(pos, end, storage, max_parsed_pos, expected);
|
||||
}
|
||||
}
|
||||
@ -351,7 +351,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
/// Опционально - может быть указан список столбцов. Он должен полностью соответствовать SELECT-у.
|
||||
/// Optional - a list of columns can be specified. It must fully comply with SELECT.
|
||||
if (s_lparen.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
@ -365,7 +365,7 @@ bool ParserCreateQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Опционально - может быть указана внутренняя ENGINE для MATERIALIZED VIEW
|
||||
/// Optional - internal ENGINE for MATERIALIZED VIEW can be specified
|
||||
engine_p.parse(pos, end, inner_storage, max_parsed_pos, expected);
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
@ -46,7 +46,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ASTPtr format;
|
||||
ASTPtr select;
|
||||
ASTPtr id;
|
||||
/// Данные для вставки
|
||||
/// Insertion data
|
||||
const char * data = nullptr;
|
||||
|
||||
ws.ignore(pos, end);
|
||||
@ -88,7 +88,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
|
||||
ws.ignore(pos, end);
|
||||
|
||||
/// Есть ли список столбцов
|
||||
/// Is there a list of columns
|
||||
if (s_lparen.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
@ -106,7 +106,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
|
||||
Pos before_select = pos;
|
||||
|
||||
/// VALUES или FORMAT или SELECT
|
||||
/// VALUES or FORMAT or SELECT
|
||||
if (s_values.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
@ -120,7 +120,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
if (!name_p.parse(pos, end, format, max_parsed_pos, expected))
|
||||
return false;
|
||||
|
||||
/// Данные начинаются после первого перевода строки, если такой есть, или после всех пробельных символов, иначе.
|
||||
/// Data starts after the first newline, if there is one, or after all the whitespace characters, otherwise.
|
||||
ParserWhiteSpaceOrComments ws_without_nl(false);
|
||||
|
||||
ws_without_nl.ignore(pos, end);
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Парсит database.table или table.
|
||||
/// Parse database.table or table.
|
||||
static bool parseDatabaseAndTable(
|
||||
ASTRenameQuery::Table & db_and_table, IParser::Pos & pos, IParser::Pos end, IParser::Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
|
@ -61,29 +61,29 @@ static bool parseDecimal(IParser::Pos & pos, IParser::Pos end, ASTSampleRatio::R
|
||||
if (exponent < 0)
|
||||
res.denominator *= exp10(-exponent);
|
||||
|
||||
/// NOTE Удаление общих степеней десяти из числителя и знаменателя - не нужно.
|
||||
/// NOTE You do not need to delete the common power of ten from the numerator and denominator.
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
/** Возможные варианты:
|
||||
/** Possible options:
|
||||
*
|
||||
* 12345
|
||||
* - целое число
|
||||
* - an integer
|
||||
*
|
||||
* 0.12345
|
||||
* .12345
|
||||
* 0.
|
||||
* - дробь в обычной десятичной записи
|
||||
* - fraction in ordinary decimal notation
|
||||
*
|
||||
* 1.23e-1
|
||||
* - дробь в инженерной десятичной записи
|
||||
* - fraction in engineering decimal notation
|
||||
*
|
||||
* 123 / 456
|
||||
* - дробь с произвольным знаменателем
|
||||
* - fraction with an ordinary denominator
|
||||
*
|
||||
* На всякий случай, в числителе и знаменателе дроби, поддерживаем предыдущие случаи.
|
||||
* Пример:
|
||||
* Just in case, in the numerator and denominator of the fraction, we support the previous cases.
|
||||
* Example:
|
||||
* 123.0 / 456e0
|
||||
*/
|
||||
bool ParserSampleRatio::parseImpl(IParser::Pos & pos, IParser::Pos end, ASTPtr & node, IParser::Pos & max_parsed_pos, Expected & expected)
|
||||
|
@ -71,7 +71,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
|
||||
ws.ignore(pos, end);
|
||||
}
|
||||
|
||||
/// FROM database.table или FROM table или FROM (subquery) или FROM tableFunction
|
||||
/// FROM database.table or FROM table or FROM (subquery) or FROM tableFunction
|
||||
if (s_from.ignore(pos, end, max_parsed_pos, expected))
|
||||
{
|
||||
ws.ignore(pos, end);
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Парсит name = value.
|
||||
/// Parse `name = value`.
|
||||
static bool parseNameValuePair(ASTSetQuery::Change & change, IParser::Pos & pos, IParser::Pos end, IParser::Pos & max_parsed_pos, Expected & expected)
|
||||
{
|
||||
ParserIdentifier name_p;
|
||||
|
Loading…
Reference in New Issue
Block a user