From b4c23f122a4bf7528e818cb1d18e54fcc2dc69f4 Mon Sep 17 00:00:00 2001 From: f1yegor Date: Sun, 2 Apr 2017 20:37:49 +0300 Subject: [PATCH] Translated comments --- dbms/src/Client/clickhouse-client.xml | 2 +- .../DataStreams/RemoteBlockInputStream.cpp | 1 - dbms/src/Interpreters/Aggregator.cpp | 256 +++++------ dbms/src/Interpreters/Cluster.cpp | 6 +- .../ClusterProxy/DescribeQueryConstructor.cpp | 8 +- .../ClusterProxy/SelectQueryConstructor.cpp | 8 +- dbms/src/Interpreters/Compiler.cpp | 20 +- dbms/src/Interpreters/Context.cpp | 72 +-- .../src/Interpreters/EmbeddedDictionaries.cpp | 10 +- dbms/src/Interpreters/ExpressionActions.cpp | 78 ++-- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 416 +++++++++--------- .../src/Interpreters/ExternalDictionaries.cpp | 8 +- .../Interpreters/InterpreterCheckQuery.cpp | 18 +- .../Interpreters/InterpreterCreateQuery.cpp | 20 +- .../src/Interpreters/InterpreterDropQuery.cpp | 10 +- dbms/src/Interpreters/InterpreterFactory.cpp | 2 +- .../Interpreters/InterpreterInsertQuery.cpp | 12 +- .../Interpreters/InterpreterSelectQuery.cpp | 246 +++++------ dbms/src/Interpreters/InterpreterSetQuery.cpp | 10 +- .../InterpreterShowTablesQuery.cpp | 6 +- dbms/src/Interpreters/Join.cpp | 62 +-- .../LogicalExpressionsOptimizer.cpp | 38 +- dbms/src/Interpreters/ProcessList.cpp | 24 +- dbms/src/Interpreters/QueryLog.cpp | 2 +- dbms/src/Interpreters/Quota.cpp | 2 +- dbms/src/Interpreters/Set.cpp | 56 +-- dbms/src/Interpreters/SetVariants.cpp | 4 +- dbms/src/Interpreters/Settings.cpp | 26 +- dbms/src/Interpreters/convertFieldToType.cpp | 12 +- .../evaluateConstantExpression.cpp | 2 +- dbms/src/Interpreters/sortBlock.cpp | 6 +- dbms/src/Interpreters/tests/hash_map.cpp | 38 +- dbms/src/Interpreters/tests/hash_map2.cpp | 18 +- .../Interpreters/tests/hash_map_string.cpp | 18 +- .../Interpreters/tests/hash_map_string_2.cpp | 4 +- .../Interpreters/tests/hash_map_string_3.cpp | 2 +- .../tests/in_join_subqueries_preprocessor.cpp | 46 +- .../tests/logical_expressions_optimizer.cpp | 8 +- dbms/src/Interpreters/tests/select_query.cpp | 2 +- dbms/src/Interpreters/tests/users.cpp | 2 +- dbms/src/Parsers/ASTAlterQuery.cpp | 2 +- dbms/src/Parsers/ASTFunction.cpp | 16 +- dbms/src/Parsers/ASTIdentifier.cpp | 2 +- dbms/src/Parsers/ASTSelectQuery.cpp | 46 +- dbms/src/Parsers/ASTWithAlias.cpp | 4 +- dbms/src/Parsers/ExpressionElementParsers.cpp | 52 +-- dbms/src/Parsers/ExpressionListParsers.cpp | 40 +- dbms/src/Parsers/IAST.cpp | 2 +- dbms/src/Parsers/ParserCreateQuery.cpp | 12 +- dbms/src/Parsers/ParserInsertQuery.cpp | 8 +- dbms/src/Parsers/ParserRenameQuery.cpp | 2 +- dbms/src/Parsers/ParserSampleRatio.cpp | 16 +- dbms/src/Parsers/ParserSelectQuery.cpp | 2 +- dbms/src/Parsers/ParserSetQuery.cpp | 2 +- 54 files changed, 893 insertions(+), 894 deletions(-) diff --git a/dbms/src/Client/clickhouse-client.xml b/dbms/src/Client/clickhouse-client.xml index 1173fafc98b..8713e1e1e44 100644 --- a/dbms/src/Client/clickhouse-client.xml +++ b/dbms/src/Client/clickhouse-client.xml @@ -1,3 +1,3 @@ - + diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 821f7b5d834..607b7913485 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -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. diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 5313b844282..d49a1a02e47 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -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(&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("_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 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(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(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()); 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 @@ -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( *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(*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> 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 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(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(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 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 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 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 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(*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; 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 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(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 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); diff --git a/dbms/src/Interpreters/Cluster.cpp b/dbms/src/Interpreters/Cluster.cpp index 527f7a00e65..4d9b007014b 100644 --- a/dbms/src/Interpreters/Cluster.cpp +++ b/dbms/src/Interpreters/Cluster.cpp @@ -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); diff --git a/dbms/src/Interpreters/ClusterProxy/DescribeQueryConstructor.cpp b/dbms/src/Interpreters/ClusterProxy/DescribeQueryConstructor.cpp index b2e9187b18b..55977637912 100644 --- a/dbms/src/Interpreters/ClusterProxy/DescribeQueryConstructor.cpp +++ b/dbms/src/Interpreters/ClusterProxy/DescribeQueryConstructor.cpp @@ -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(stream); return std::make_shared(materialized_stream, toBlockExtraInfo(address)); diff --git a/dbms/src/Interpreters/ClusterProxy/SelectQueryConstructor.cpp b/dbms/src/Interpreters/ClusterProxy/SelectQueryConstructor.cpp index d59764c44a4..e6a3c2db774 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectQueryConstructor.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectQueryConstructor.cpp @@ -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(stream); } diff --git a/dbms/src/Interpreters/Compiler.cpp b/dbms/src/Interpreters/Compiler.cpp index b5ccc55b53e..c73061550ce 100644 --- a/dbms/src/Interpreters/Compiler.cpp +++ b/dbms/src/Interpreters/Compiler.cpp @@ -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); diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index e1bdb38a17a..d514aa924a8 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -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 embedded_dictionaries; /// Metrica's dictionaeis. Have lazy initialization. mutable std::shared_ptr 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; /// Used for dynamic compilation of queries' parts if it necessary. std::unique_ptr query_log; /// Used to log queries. std::shared_ptr 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 compression_method_selector; std::unique_ptr 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; 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; } diff --git a/dbms/src/Interpreters/EmbeddedDictionaries.cpp b/dbms/src/Interpreters/EmbeddedDictionaries.cpp index bdc3207eb2d..06f6b4b6293 100644 --- a/dbms/src/Interpreters/EmbeddedDictionaries.cpp +++ b/dbms/src/Interpreters/EmbeddedDictionaries.cpp @@ -46,11 +46,11 @@ bool EmbeddedDictionaries::reloadDictionary(MultiVersion & 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."); diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 4816a54a3bf..08f01e925d9 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -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 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(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 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(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(); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 4728300cae2..1d50d1112d4 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -170,23 +170,23 @@ void ExpressionAnalyzer::init() select_query = typeid_cast(ast.get()); - /// В зависимости от профиля пользователя проверить наличие прав на выполнение - /// распределённых подзапросов внутри секций IN или JOIN и обработать эти подзапросы. + /// Depending on the user's profile, check for the execution rights + /// distributed subqueries inside the IN or JOIN sections and process these subqueries. InJoinSubqueriesPreprocessor(context).process(select_query); - /// Оптимизирует логические выражения. + /// Optimizes logical expressions. LogicalExpressionsOptimizer(select_query, settings).perform(); - /// Создаёт словарь aliases: alias -> ASTPtr + /// Creates a dictionary `aliases`: alias -> ASTPtr addASTAliases(ast); /// Common subexpression elimination. Rewrite rules. normalizeTree(); - /// ALIAS столбцы не должны подставляться вместо ASTAsterisk, добавим их теперь, после normalizeTree. + /// ALIAS columns should not be substituted for ASTAsterisk, we will add them now, after normalizeTree. addAliasColumns(); - /// Выполнение скалярных подзапросов - замена их на значения-константы. + /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(); /// Optimize if with constant condition after constats are substituted instead of sclalar subqueries @@ -195,7 +195,7 @@ void ExpressionAnalyzer::init() /// GROUP BY injective function elimination. optimizeGroupBy(); - /// Удалить из ORDER BY повторяющиеся элементы. + /// Remove duplicate items from ORDER BY. optimizeOrderBy(); // Remove duplicated elements from LIMIT BY clause. @@ -204,20 +204,20 @@ void ExpressionAnalyzer::init() /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(); - /// Удалить ненужное из списка columns. Создать unknown_required_columns. Сформировать columns_added_by_join. + /// Delete the unnecessary from `columns` list. Create `unknown_required_columns`. Form `columns_added_by_join`. collectUsedColumns(); - /// external_tables, subqueries_for_sets для глобальных подзапросов. - /// Заменяет глобальные подзапросы на сгенерированные имена временных таблиц, которые будут отправлены на удалённые серверы. + /// external_tables, subqueries_for_sets for global subqueries. + /// Replaces global subqueries with the generated names of temporary tables that will be sent to remote servers. initGlobalSubqueriesAndExternalTables(); /// has_aggregation, aggregation_keys, aggregate_descriptions, aggregated_columns. - /// Этот анализ надо провести после обработки глобальных подзапросов, потому что в противном случае, - /// если агрегатная функция содержит глобальный подзапрос, то метод analyzeAggregation сохранит - /// в aggregate_descriptions информацию о параметрах этой агрегатной функции, среди которых окажется - /// глобальный подзапрос. Затем при вызове метода initGlobalSubqueriesAndExternalTables, этот - /// глобальный подзапрос будет заменён на временную таблицу, в результате чего aggregate_descriptions - /// будет содержать устаревшую информацию, что приведёт к ошибке при выполнении запроса. + /// This analysis should be performed after processing global subqueries, because otherwise, + /// if the aggregate function contains a global subquery, then `analyzeAggregation` method will save + /// in `aggregate_descriptions` the information about the parameters of this aggregate function, among which + /// global subquery. Then, when you call `initGlobalSubqueriesAndExternalTables` method, this + /// the global subquery will be replaced with a temporary table, resulting in aggregate_descriptions + /// will contain out-of-date information, which will lead to an error when the query is executed. analyzeAggregation(); } @@ -317,11 +317,11 @@ void ExpressionAnalyzer::optimizeIfWithConstantConditionImpl(ASTPtr & current_as void ExpressionAnalyzer::analyzeAggregation() { - /** Найдем ключи агрегации (aggregation_keys), информацию об агрегатных функциях (aggregate_descriptions), - * а также набор столбцов, получаемых после агрегации, если она есть, - * или после всех действий, которые обычно выполняются до агрегации (aggregated_columns). + /** Find aggregation keys (aggregation_keys), information about aggregate functions (aggregate_descriptions), + * as well as a set of columns obtained after the aggregation, if any, + * or after all the actions that are usually performed before aggregation (aggregated_columns). * - * Всё, что ниже (составление временных ExpressionActions) - только в целях анализа запроса (вывода типов). + * Everything below (compiling temporary ExpressionActions) - only for the purpose of query analysis (type output). */ if (select_query && (select_query->group_expression_list || select_query->having_expression)) @@ -422,33 +422,33 @@ void ExpressionAnalyzer::analyzeAggregation() void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables() { - /// Добавляет уже существующие внешние таблицы (не подзапросы) в словарь external_tables. + /// Adds existing external tables (not subqueries) to the external_tables dictionary. findExternalTables(ast); - /// Преобразует GLOBAL-подзапросы во внешние таблицы; кладёт их в словарь external_tables: name -> StoragePtr. + /// Converts GLOBAL subqueries to external tables; Puts them into the external_tables dictionary: name -> StoragePtr. initGlobalSubqueries(ast); } void ExpressionAnalyzer::initGlobalSubqueries(ASTPtr & ast) { - /// Рекурсивные вызовы. Не опускаемся в подзапросы. + /// Recursive calls. We do not go into subqueries. for (auto & child : ast->children) if (!typeid_cast(child.get())) initGlobalSubqueries(child); - /// Действия, выполняемые снизу вверх. + /// Bottom-up actions. if (ASTFunction * node = typeid_cast(ast.get())) { - /// Для GLOBAL IN. + /// For GLOBAL IN. if (do_global && (node->name == "globalIn" || node->name == "globalNotIn")) addExternalStorage(node->arguments->children.at(1)); } else if (ASTTablesInSelectQueryElement * node = typeid_cast(ast.get())) { - /// Для GLOBAL JOIN. + /// For GLOBAL JOIN. if (do_global && node->table_join && static_cast(*node->table_join).locality == ASTTableJoin::Locality::Global) addExternalStorage(node->table_expression); @@ -458,11 +458,11 @@ void ExpressionAnalyzer::initGlobalSubqueries(ASTPtr & ast) void ExpressionAnalyzer::findExternalTables(ASTPtr & ast) { - /// Обход снизу. Намеренно опускаемся в подзапросы. + /// Traverse from the bottom. Intentionally go into subqueries. for (auto & child : ast->children) findExternalTables(child); - /// Если идентификатор типа таблица + /// If table type identifier StoragePtr external_storage; if (ASTIdentifier * node = typeid_cast(ast.get())) @@ -478,7 +478,7 @@ static std::shared_ptr interpretSubquery( void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression) { - /// При нераспределённых запросах, создание временных таблиц не имеет смысла. + /// With nondistributed queries, creating temporary tables does not make sense. if (!(storage && storage->isRemote())) return; @@ -515,12 +515,12 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t if (table_name) { - /// Если это уже внешняя таблица, ничего заполять не нужно. Просто запоминаем ее наличие. + /// If this is already an external table, you do not need to add anything. Just remember its presence. if (external_tables.end() != external_tables.find(static_cast(*table_name).name)) return; } - /// Сгенерируем имя для внешней таблицы. + /// Generate the name for the external table. String external_table_name = "_data" + toString(external_table_id); while (external_tables.count(external_table_name)) { @@ -535,27 +535,27 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t StoragePtr external_storage = StorageMemory::create(external_table_name, columns); - /** Есть два способа выполнения распределённых GLOBAL-подзапросов. + /** There are two ways to perform distributed GLOBAL subqueries. * - * Способ push: - * Данные подзапроса отправляются на все удалённые серверы, где они затем используются. - * Для этого способа, данные отправляются в виде "внешних таблиц" и будут доступны на каждом удалённом сервере по имени типа _data1. - * Заменяем в запросе подзапрос на это имя. + * "push" method: + * Subquery data is sent to all remote servers, where they are then used. + * For this method, the data is sent in the form of "external tables" and will be available on each remote server by the name of the type _data1. + * Replace in the query a subquery for this name. * - * Способ pull: - * Удалённые серверы скачивают данные подзапроса с сервера-инициатора запроса. - * Для этого способа, заменяем подзапрос на другой подзапрос вида (SELECT * FROM remote('host:port', _query_QUERY_ID, _data1)) - * Этот подзапрос, по факту, говорит - "надо скачать данные оттуда". + * "pull" method: + * Remote servers download the subquery data from the request initiating server. + * For this method, replace the subquery with another subquery of the form (SELECT * FROM remote ('host: port', _query_QUERY_ID, _data1)) + * This subquery, in fact, says - "you need to download data from there." * - * Способ pull имеет преимущество, потому что в нём удалённый сервер может решить, что ему не нужны данные и не скачивать их в таких случаях. + * The "pull" method takes precedence, because in it a remote server can decide that it does not need data and does not download it in such cases. */ if (settings.global_subqueries_method == GlobalSubqueriesMethod::PUSH) { - /** Заменяем подзапрос на имя временной таблицы. - * Именно в таком виде, запрос отправится на удалённый сервер. - * На удалённый сервер отправится эта временная таблица, и на его стороне, - * вместо выполнения подзапроса, надо будет просто из неё прочитать. + /** We replace the subquery with the name of the temporary table. + * It is in this form, the request will go to the remote server. + * This temporary table will go to the remote server, and on its side, + * instead of doing a subquery, you just need to read it. */ auto database_and_table_name = std::make_shared(StringRange(), external_table_name, ASTIdentifier::Table); @@ -619,10 +619,10 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t subqueries_for_sets[external_table_name].source_sample = interpreter->getSampleBlock(); subqueries_for_sets[external_table_name].table = external_storage; - /** NOTE Если было написано IN tmp_table - существующая временная (но не внешняя) таблица, - * то здесь будет создана новая временная таблица (например, _data1), - * и данные будут затем в неё скопированы. - * Может быть, этого можно избежать. + /** NOTE If it was written IN tmp_table - the existing temporary (but not external) table, + * then a new temporary table will be created (for example, _data1), + * and the data will then be copied to it. + * Maybe this can be avoided. */ } @@ -634,17 +634,17 @@ NamesAndTypesList::iterator ExpressionAnalyzer::findColumn(const String & name, } -/// ignore_levels - алиасы в скольки верхних уровнях поддерева нужно игнорировать. -/// Например, при ignore_levels=1 ast не может быть занесен в словарь, но его дети могут. +/// ignore_levels - aliases in how many upper levels of the subtree should be ignored. +/// For example, with ignore_levels=1 ast can not be put in the dictionary, but its children can. void ExpressionAnalyzer::addASTAliases(ASTPtr & ast, int ignore_levels) { - /// Обход снизу-вверх. Не опускаемся в подзапросы. + /// Bottom-up traversal. We do not go into subqueries. for (auto & child : ast->children) { int new_ignore_levels = std::max(0, ignore_levels - 1); - /// Алиасы верхнего уровня в секции ARRAY JOIN имеют особый смысл, их добавлять не будем - /// (пропустим сам expression list и его детей). + /// The top-level aliases in the ARRAY JOIN section have a special meaning, we will not add them + /// (skip the expression list itself and its children). if (typeid_cast(ast.get())) new_ignore_levels = 3; @@ -699,9 +699,9 @@ void ExpressionAnalyzer::normalizeTree() } -/// finished_asts - уже обработанные вершины (и на что они заменены) -/// current_asts - вершины в текущем стеке вызовов этого метода -/// current_alias - алиас, повешенный на предка ast (самого глубокого из предков с алиасами) +/// finished_asts - already processed vertices (and by what they replaced) +/// current_asts - vertices in the current call stack of this method +/// current_alias - the alias referencing to the ancestor of ast (the deepest ancestor with aliases) void ExpressionAnalyzer::normalizeTreeImpl( ASTPtr & ast, MapOfASTs & finished_asts, SetOfASTs & current_asts, std::string current_alias, size_t level) { @@ -721,14 +721,14 @@ void ExpressionAnalyzer::normalizeTreeImpl( if (!my_alias.empty()) current_alias = my_alias; - /// rewrite правила, которые действуют при обходе сверху-вниз. + /// rewrite rules that act when you go from top to bottom. bool replaced = false; ASTFunction * func_node = typeid_cast(ast.get()); if (func_node) { - /** Нет ли в таблице столбца, название которого полностью совпадает с записью функции? - * Например, в таблице есть столбец "domain(URL)", и мы запросили domain(URL). + /** Is there a column in the table whose name fully matches the function entry? + * For example, in the table there is a column "domain(URL)", and we requested domain(URL). */ String function_string = func_node->getColumnName(); NamesAndTypesList::const_iterator it = findColumn(function_string); @@ -739,7 +739,7 @@ void ExpressionAnalyzer::normalizeTreeImpl( replaced = true; } - /// Может быть указано IN t, где t - таблица, что равносильно IN (SELECT * FROM t). + /// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`. if (functionIsInOrGlobalInOperator(func_node->name)) if (ASTIdentifier * right = typeid_cast(func_node->arguments->children.at(1).get())) right->kind = ASTIdentifier::Table; @@ -767,16 +767,16 @@ void ExpressionAnalyzer::normalizeTreeImpl( { if (node->kind == ASTIdentifier::Column) { - /// Если это алиас, но не родительский алиас (чтобы работали конструкции вроде "SELECT column+1 AS column"). + /// If it is an alias, but not a parent alias (for constructs like "SELECT column + 1 AS column"). Aliases::const_iterator jt = aliases.find(node->name); if (jt != aliases.end() && current_alias != node->name) { - /// Заменим его на соответствующий узел дерева. + /// Let's replace it with the corresponding tree node. if (current_asts.count(jt->second.get())) throw Exception("Cyclic aliases", ErrorCodes::CYCLIC_ALIASES); if (!my_alias.empty() && my_alias != jt->second->getAliasOrColumnName()) { - /// В конструкции вроде "a AS b", где a - алиас, нужно перевесить алиас b на результат подстановки алиаса a. + /// In a construct like "a AS b", where a is an alias, you must set alias b to the result of substituting alias a. ast = jt->second->clone(); ast->setAlias(my_alias); } @@ -791,7 +791,7 @@ void ExpressionAnalyzer::normalizeTreeImpl( } else if (ASTExpressionList * node = typeid_cast(ast.get())) { - /// Заменим * на список столбцов. + /// Replace * with a list of columns. ASTs & asts = node->children; for (int i = static_cast(asts.size()) - 1; i >= 0; --i) { @@ -821,7 +821,7 @@ void ExpressionAnalyzer::normalizeTreeImpl( } } - /// Если заменили корень поддерева вызовемся для нового корня снова - на случай, если алиас заменился на алиас. + /// If we replace the root of the subtree, we will be called again for the new root, in case the alias is replaced by an alias. if (replaced) { normalizeTreeImpl(ast, finished_asts, current_asts, current_alias, level + 1); @@ -831,13 +831,13 @@ void ExpressionAnalyzer::normalizeTreeImpl( return; } - /// Рекурсивные вызовы. Не опускаемся в подзапросы. - /// Также не опускаемся в левый аргумент лямбда-выражений, чтобы не заменять формальные параметры - /// по алиасам в выражениях вида 123 AS x, arrayMap(x -> 1, [2]). + /// Recurring calls. Don't go into subqueries. + /// We also do not go to the left argument of lambda expressions, so as not to replace the formal parameters + /// on aliases in expressions of the form 123 AS x, arrayMap(x -> 1, [2]). if (func_node && func_node->name == "lambda") { - /// Пропускаем первый аргумент. Также предполагаем, что у функции lambda не может быть parameters. + /// We skip the first argument. We also assume that the lambda function can not have parameters. for (size_t i = 1, size = func_node->arguments->children.size(); i < size; ++i) { auto & child = func_node->arguments->children[i]; @@ -861,7 +861,7 @@ void ExpressionAnalyzer::normalizeTreeImpl( } } - /// Если секция WHERE или HAVING состоит из одного алиаса, ссылку нужно заменить не только в children, но и в where_expression и having_expression. + /// If the WHERE clause or HAVING consists of a single alias, the reference must be replaced not only in children, but also in where_expression and having_expression. if (ASTSelectQuery * select = typeid_cast(ast.get())) { if (select->prewhere_expression) @@ -872,7 +872,7 @@ void ExpressionAnalyzer::normalizeTreeImpl( normalizeTreeImpl(select->having_expression, finished_asts, current_asts, current_alias, level + 1); } - /// Действия, выполняемые снизу вверх. + /// Actions to be performed from the bottom up. if (ASTFunction * node = typeid_cast(ast.get())) { @@ -927,7 +927,7 @@ void ExpressionAnalyzer::executeScalarSubqueries() { for (auto & child : ast->children) { - /// Не опускаемся в FROM, JOIN, UNION. + /// Do not go to FROM, JOIN, UNION. if (!typeid_cast(child.get()) && child.get() != select_query->next_union_all.get()) { @@ -957,21 +957,21 @@ static ASTPtr addTypeConversion(std::unique_ptr && ast, const String void ExpressionAnalyzer::executeScalarSubqueriesImpl(ASTPtr & ast) { - /** Заменяем подзапросы, возвращающие ровно одну строку - * ("скалярные" подзапросы) на соответствующие константы. + /** Replace subqueries that return exactly one row + * ("scalar" subqueries) to the corresponding constants. * - * Если подзапрос возвращает более одного столбца, то он заменяется на кортеж констант. + * If the subquery returns more than one column, it is replaced by a tuple of constants. * - * Особенности: + * Features * - * Замена происходит во время анализа запроса, а не во время основной стадии выполнения. - * Это значит, что не будет работать индикатор прогресса во время выполнения этих запросов, - * а также такие запросы нельзя будет прервать. + * A replacement occurs during query analysis, and not during the main runtime. + * This means that the progress indicator will not work during the execution of these requests, + * and also such queries can not be aborted. * - * Зато результат запросов может быть использован для индекса в таблице. + * But the query result can be used for the index in the table. * - * Скалярные подзапросы выполняются на сервере-инициаторе запроса. - * На удалённые серверы запрос отправляется с уже подставленными константами. + * Scalar subqueries are executed on the request-initializer server. + * The request is sent to remote servers with already substituted constants. */ if (ASTSubquery * subquery = typeid_cast(ast.get())) @@ -1149,12 +1149,12 @@ void ExpressionAnalyzer::optimizeGroupBy() if (group_exprs.empty()) { - /** Нельзя полностью убирать GROUP BY. Потому что если при этом даже агрегатных функций не было, то получится, что не будет агрегации. - * Вместо этого оставим GROUP BY const. - * Далее см. удаление констант в методе analyzeAggregation. + /** You can not completely remove GROUP BY. Because if there were no aggregate functions, then it turns out that there will be no aggregation. + * Instead, leave `GROUP BY const`. + * Next, see deleting the constants in the analyzeAggregation method. */ - /// Нужно вставить константу, которая не является именем столбца таблицы. Такой случай редкий, но бывает. + /// You must insert a constant that is not the name of the column in the table. Such a case is rare, but it happens. UInt64 unused_column = 0; String unused_column_name = toString(unused_column); @@ -1176,7 +1176,7 @@ void ExpressionAnalyzer::optimizeOrderBy() if (!(select_query && select_query->order_expression_list)) return; - /// Уникализируем условия сортировки. + /// Make unique sorting conditions. using NameAndLocale = std::pair; std::set elems_set; @@ -1245,7 +1245,7 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sampl } catch (const DB::Exception & e) { - /// в sample_block нет колонок, которые добаляет getActions + /// in `sample_block` there are no columns that add `getActions` if (e.code() != ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK) throw; } @@ -1257,25 +1257,25 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sampl static std::shared_ptr interpretSubquery( ASTPtr & subquery_or_table_name, const Context & context, size_t subquery_depth, const Names & required_columns) { - /// Подзапрос или имя таблицы. Имя таблицы аналогично подзапросу SELECT * FROM t. + /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. const ASTSubquery * subquery = typeid_cast(subquery_or_table_name.get()); const ASTIdentifier * table = typeid_cast(subquery_or_table_name.get()); if (!subquery && !table) throw Exception("IN/JOIN supports only SELECT subqueries.", ErrorCodes::BAD_ARGUMENTS); - /** Для подзапроса в секции IN/JOIN не действуют ограничения на максимальный размер результата. - * Так как результат этого поздапроса - ещё не результат всего запроса. - * Вместо этого работают ограничения + /** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result. + * Because the result of this query is not the result of the entire query. + * Constraints work instead * max_rows_in_set, max_bytes_in_set, set_overflow_mode, * max_rows_in_join, max_bytes_in_join, join_overflow_mode, - * которые проверяются отдельно (в объектах Set, Join). + * which are checked separately (in the Set, Join objects). */ 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 instead of the whole query). subquery_settings.extremes = 0; subquery_context.setSettings(subquery_settings); @@ -1306,11 +1306,11 @@ static std::shared_ptr interpretSubquery( { query = subquery->children.at(0); - /** В подзапросе могут быть указаны столбцы с одинаковыми именами. Например, SELECT x, x FROM t - * Это плохо, потому что результат такого запроса нельзя сохранить в таблицу, потому что в таблице не может быть одноимённых столбцов. - * Сохранение в таблицу требуется для GLOBAL-подзапросов. + /** Columns with the same name can be specified in a subquery. For example, SELECT x, x FROM t + * This is bad, because the result of such a query can not be saved to the table, because the table can not have the same name columns. + * Saving to the table is required for GLOBAL subqueries. * - * Чтобы избежать такой ситуации, будем переименовывать одинаковые столбцы. + * To avoid this situation, we will rename the same columns. */ std::set all_column_names; @@ -1332,7 +1332,7 @@ static std::shared_ptr interpretSubquery( ++i; name = name + "_" + toString(i); - expr = expr->clone(); /// Отменяет склейку одинаковых выражений в дереве. + expr = expr->clone(); /// Cancels fuse of the same expressions in the tree. expr->setAlias(name); all_column_names.insert(name); @@ -1353,28 +1353,28 @@ static std::shared_ptr interpretSubquery( void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) { - /** Нужно преобразовать правый аргумент в множество. - * Это может быть имя таблицы, значение, перечисление значений или подзапрос. - * Перечисление значений парсится как функция tuple. + /** You need to convert the right argument to a set. + * This can be a table name, a value, a value enumeration, or a subquery. + * The enumeration of values is parsed as a function `tuple`. */ IAST & args = *node->arguments; ASTPtr & arg = args.children.at(1); - /// Уже преобразовали. + /// Already converted. if (typeid_cast(arg.get())) return; - /// Если подзапрос или имя таблицы для SELECT. + /// If the subquery or table name for SELECT. ASTIdentifier * identifier = typeid_cast(arg.get()); if (typeid_cast(arg.get()) || identifier) { - /// Получаем поток блоков для подзапроса. Создаём Set и кладём на место подзапроса. + /// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery. String set_id = arg->getColumnName(); auto ast_set = std::make_shared(set_id); ASTPtr ast_set_ptr = ast_set; - /// Особый случай - если справа оператора IN указано имя таблицы, при чём, таблица имеет тип Set (заранее подготовленное множество). - /// TODO В этом синтаксисе не поддерживается указание имени БД. + /// A special case is if the name of the table is specified on the right side of the IN statement, and the table has the type Set (a previously prepared set). + /// TODO This syntax does not support the specification of the database name. if (identifier) { StoragePtr table = context.tryGetTable("", identifier->name); @@ -1395,7 +1395,7 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) SubqueryForSet & subquery_for_set = subqueries_for_sets[set_id]; - /// Если уже создали Set с таким же подзапросом/таблицей. + /// If you already created a Set with the same subquery / table. if (subquery_for_set.set) { ast_set->set = subquery_for_set.set; @@ -1405,10 +1405,10 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) ast_set->set = std::make_shared(settings.limits); - /** Для GLOBAL IN-ов происходит следующее: - * - в функции addExternalStorage подзапрос IN (SELECT ...) заменяется на IN _data1, - * в объекте subquery_for_set выставляется этот подзапрос в качестве source и временная таблица _data1 в качестве table. - * - в этой функции видно выражение IN _data1. + /** The following happens for GLOBAL INs: + * - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1, + * in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table. + * - this function shows the expression IN_data1. */ if (!subquery_for_set.source) { @@ -1417,31 +1417,31 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) [interpreter]() mutable { return interpreter->execute().in; }); subquery_for_set.source_sample = interpreter->getSampleBlock(); - /** Зачем используется LazyBlockInputStream? + /** Why is LazyBlockInputStream used? * - * Дело в том, что при обработке запроса вида + * The fact is that when processing a request of the form * SELECT ... FROM remote_test WHERE column GLOBAL IN (subquery), - * если распределённая таблица remote_test содержит в качестве одного из серверов localhost, - * то запрос будет ещё раз интерпретирован локально (а не отправлен по TCP, как в случае удалённого сервера). + * if the distributed remote_test table contains localhost as one of the servers, + * the request will be interpreted locally again (and not sent over TCP, as in the case of a remote server). * - * Конвейер выполнения запроса будет такой: + * The query execution pipeline will be: * CreatingSets - * выполнение подзапроса subquery, заполнение временной таблицы _data1 (1) + * subquery execution, filling the temporary table with _data1 (1) * CreatingSets - * чтение из таблицы _data1, создание множества (2) - * чтение из таблицы, подчинённой remote_test. + * reading from the table _data1, creating the set (2) + * read from the table subordinate to remote_test. * - * (Вторая часть конвейера под CreatingSets - это повторная интерпретация запроса внутри StorageDistributed, - * запрос отличается тем, что имя БД и таблицы заменены на подчинённые, а также подзапрос заменён на _data1.) + * (The second part of the pipeline under CreateSets is a reinterpretation of the request inside StorageDistributed, + * the query differs in that the database name and tables are replaced with subordinates, and the subquery is replaced with _data1.) * - * Но при создании конвейера, при создании источника (2), будет обнаружено, что таблица _data1 пустая - * (потому что запрос ещё не начал выполняться), и будет возвращён в качестве источника пустой источник. - * И затем, при выполнении запроса, на шаге (2), будет создано пустое множество. + * But when creating the pipeline, when creating the source (2), it will be found that the _data1 table is empty + * (because the query has not started yet), and empty source will be returned as the source. + * And then, when the query is executed, an empty set will be created in step (2). * - * Поэтому, мы делаем инициализацию шага (2) ленивой - * - чтобы она произошла только после выполнения шага (1), на котором нужная таблица будет заполнена. + * Therefore, we make the initialization of step (2) lazy + * - so that it does not occur until step (1) is completed, on which the table will be populated. * - * Замечание: это решение не очень хорошее, надо подумать лучше. + * Note: this solution is not very good, you need to think better. */ } @@ -1450,12 +1450,12 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block) } else { - /// Явное перечисление значений в скобках. + /// An explicit enumeration of values in parentheses. makeExplicitSet(node, sample_block, false); } } -/// Случай явного перечисления значений. +/// The case of an explicit enumeration of values. void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sample_block, bool create_ordered_set) { IAST & args = *node->arguments; @@ -1497,7 +1497,7 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl set_element_types.push_back(left_type); } - /// Отличим случай x in (1, 2) от случая x in 1 (он же x in (1)). + /// The case `x in (1, 2)` distinguishes from the case `x in 1` (also `x in (1)`). bool single_value = false; ASTPtr elements_ast = arg; @@ -1507,12 +1507,12 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl { if (set_func->arguments->children.empty()) { - /// Пустое множество. + /// Empty set. elements_ast = set_func->arguments; } else { - /// Отличим случай (x, y) in ((1, 2), (3, 4)) от случая (x, y) in (1, 2). + /// Distinguish the case `(x, y) in ((1, 2), (3, 4))` from the case `(x, y) in (1, 2)`. ASTFunction * any_element = typeid_cast(set_func->arguments->children.at(0).get()); if (set_element_types.size() >= 2 && (!any_element || any_element->name != "tuple")) single_value = true; @@ -1564,11 +1564,11 @@ static String getUniqueName(const Block & block, const String & prefix) } -/** Для getActionsImpl. - * Стек из ExpressionActions, соответствующих вложенным лямбда-выражениям. - * Новое действие нужно добавлять на самый высокий возможный уровень. - * Например, в выражении "select arrayMap(x -> x + column1 * column2, array1)" - * вычисление произведения нужно делать вне лямбда-выражения (оно не зависит от x), а вычисление суммы - внутри (зависит от x). +/** For getActionsImpl. + * A stack of ExpressionActions corresponding to nested lambda expressions. + * The new action should be added to the highest possible level. + * For example, in the expression "select arrayMap(x -> x + column1 * column2, array1)" + * calculation of the product must be done outside the lambda expression (it does not depend on x), and the calculation of the sum is inside (depends on x). */ struct ExpressionAnalyzer::ScopeStack { @@ -1695,20 +1695,20 @@ void ExpressionAnalyzer::getArrayJoinedColumns() getArrayJoinedColumnsImpl(ast); - /// Если результат ARRAY JOIN не используется, придется все равно по-ARRAY-JOIN-ить какой-нибудь столбец, - /// чтобы получить правильное количество строк. + /// If the result of ARRAY JOIN is not used, it is necessary to ARRAY-JOIN any column, + /// to get the correct number of rows. if (array_join_result_to_source.empty()) { ASTPtr expr = select_query->array_join_expression_list()->children.at(0); String source_name = expr->getColumnName(); String result_name = expr->getAliasOrColumnName(); - /// Это массив. + /// This is an array. if (!typeid_cast(expr.get()) || findColumn(source_name, columns) != columns.end()) { array_join_result_to_source[result_name] = source_name; } - else /// Это вложенная таблица. + else /// This is a nested table. { bool found = false; for (const auto & column_name_type : columns) @@ -1730,7 +1730,7 @@ void ExpressionAnalyzer::getArrayJoinedColumns() } -/// Заполняет array_join_result_to_source: по каким столбцам-массивам размножить, и как их после этого назвать. +/// Fills the array_join_result_to_source: on which columns-arrays to replicate, and how to call them after that. void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast) { if (typeid_cast(ast.get())) @@ -1744,20 +1744,20 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast) if (array_join_alias_to_name.count(node->name)) { - /// Был написан ARRAY JOIN со столбцом-массивом. Пример: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1 + /// ARRAY JOIN was written with an array column. Example: SELECT K1 FROM ... ARRAY JOIN ParsedParams.Key1 AS K1 array_join_result_to_source[node->name] = array_join_alias_to_name[node->name]; /// K1 -> ParsedParams.Key1 } else if (array_join_alias_to_name.count(table_name)) { - /// Был написан ARRAY JOIN с вложенной таблицей. Пример: SELECT PP.Key1 FROM ... ARRAY JOIN ParsedParams AS PP + /// ARRAY JOIN was written with a nested table. Example: SELECT PP.KEY1 FROM ... ARRAY JOIN ParsedParams AS PP String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1 array_join_result_to_source[node->name] /// PP.Key1 -> ParsedParams.Key1 = DataTypeNested::concatenateNestedName(array_join_alias_to_name[table_name], nested_column); } else if (array_join_name_to_alias.count(table_name)) { - /** Пример: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP. - * То есть, в запросе используется исходный массив, размноженный по самому себе. + /** Example: SELECT ParsedParams.Key1 FROM ... ARRAY JOIN ParsedParams AS PP. + * That is, the query uses the original array, replicated by itself. */ String nested_column = DataTypeNested::extractNestedColumnName(node->name); /// Key1 @@ -1778,7 +1778,7 @@ void ExpressionAnalyzer::getArrayJoinedColumnsImpl(ASTPtr ast) void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack) { - /// Если результат вычисления уже есть в блоке. + /// If the result of the calculation already exists in the block. if ((typeid_cast(ast.get()) || typeid_cast(ast.get())) && actions_stack.getSampleBlock().has(ast->getColumnName())) return; @@ -1788,8 +1788,8 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl std::string name = node->getColumnName(); if (!only_consts && !actions_stack.getSampleBlock().has(name)) { - /// Запрошенного столбца нет в блоке. - /// Если такой столбец есть в таблице, значит пользователь наверно забыл окружить его агрегатной функцией или добавить в GROUP BY. + /// The requested column is not in the block. + /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. bool found = false; for (const auto & column_name_type : columns) @@ -1806,7 +1806,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl if (node->kind == ASTFunction::LAMBDA_EXPRESSION) throw Exception("Unexpected lambda expression", ErrorCodes::UNEXPECTED_EXPRESSION); - /// Функция arrayJoin. + /// Function arrayJoin. if (node->kind == ASTFunction::ARRAY_JOIN) { if (node->arguments->children.size() != 1) @@ -1832,18 +1832,18 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl { if (!no_subqueries) { - /// Найдем тип первого аргумента (потом getActionsImpl вызовется для него снова и ни на что не повлияет). + /// Let's find the type of the first argument (then getActionsImpl will be called again and will not affect anything). getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack); - /// Превратим tuple или подзапрос в множество. + /// Transform tuple or subquery into a set. makeSet(node, actions_stack.getSampleBlock()); } else { if (!only_consts) { - /// Мы в той части дерева, которую не собираемся вычислять. Нужно только определить типы. - /// Не будем выполнять подзапросы и составлять множества. Вставим произвольный столбец правильного типа. + /// We are in the part of the tree that we are not going to compute. You just need to define types. + /// Do not subquery and create sets. We insert an arbitrary column of the correct type. ColumnWithTypeAndName fake_column; fake_column.name = node->getColumnName(); fake_column.type = std::make_shared(); @@ -1854,8 +1854,8 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl } } - /// Особая функция indexHint. Всё, что внутри неё не вычисляется - /// (а используется только для анализа индекса, см. PKCondition). + /// A special function `indexHint`. Everything that is inside it is not calculated + /// (and is used only for index analysis, see PKCondition). if (node->name == "indexHint") { actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( @@ -1869,7 +1869,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl DataTypes argument_types; bool arguments_present = true; - /// Если у функции есть аргумент-лямбда-выражение, нужно определить его тип до рекурсивного вызова. + /// If the function has an argument-lambda expression, you need to determine its type before the recursive call. bool has_lambda_arguments = false; for (auto & child : node->arguments->children) @@ -1878,7 +1878,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl ASTSet * set = typeid_cast(child.get()); if (lambda && lambda->name == "lambda") { - /// Если аргумент - лямбда-выражение, только запомним его примерный тип. + /// If the argument is a lambda expression, just remember its approximate type. if (lambda->arguments->children.size() != 2) throw Exception("lambda requires two arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); @@ -1889,7 +1889,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl has_lambda_arguments = true; argument_types.emplace_back(std::make_shared(DataTypes(lambda_args_tuple->arguments->children.size()))); - /// Выберем название в следующем цикле. + /// Select the name in the next cycle. argument_names.emplace_back(); } else if (set) @@ -1897,8 +1897,8 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl ColumnWithTypeAndName column; column.type = std::make_shared(); - /// Если аргумент - множество, заданное перечислением значений, дадим ему уникальное имя, - /// чтобы множества с одинаковой записью не склеивались (у них может быть разный тип). + /// If the argument is a set given by an enumeration of values, give it a unique name, + /// so that sets with the same record do not fuse together (they can have different types). if (set->is_explicit) column.name = getUniqueName(actions_stack.getSampleBlock(), "__set"); else @@ -1916,7 +1916,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl } else { - /// Если аргумент не лямбда-выражение, вызовемся рекурсивно и узнаем его тип. + /// If the argument is not a lambda expression, call it recursively and find out its type. getActionsImpl(child, no_subqueries, only_consts, actions_stack); std::string name = child->getColumnName(); if (actions_stack.getSampleBlock().has(name)) @@ -1947,7 +1947,7 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl { function->getLambdaArgumentTypes(argument_types); - /// Вызовемся рекурсивно для лямбда-выражений. + /// Call recursively for lambda expressions. for (size_t i = 0; i < node->arguments->children.size(); ++i) { ASTPtr child = node->arguments->children[i]; @@ -1985,8 +1985,8 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl if (findColumn(captured[j], lambda_arguments) == lambda_arguments.end()) additional_requirements.push_back(captured[j]); - /// Не можем дать название getColumnName(), - /// потому что оно не однозначно определяет выражение (типы аргументов могут быть разными). + /// We can not name `getColumnName()`, + /// because it does not uniquely define the expression (the types of arguments can be different). argument_names[i] = getUniqueName(actions_stack.getSampleBlock(), "__lambda"); ColumnWithTypeAndName lambda_column; @@ -2036,14 +2036,14 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr & actions) { - /// Внутри WHERE и PREWHERE не может быть агрегатных функций. + /// There can not be aggregate functions inside the WHERE and PREWHERE. if (select_query && (ast.get() == select_query->where_expression.get() || ast.get() == select_query->prewhere_expression.get())) { assertNoAggregates(ast, "in WHERE or PREWHERE"); return; } - /// Если мы анализируем не запрос SELECT, а отдельное выражение, то в нём не может быть агрегатных функций. + /// If we are not analyzing a SELECT query, but a separate expression, then there can not be aggregate functions in it. if (!select_query) { assertNoAggregates(ast, "in wrong place"); @@ -2057,7 +2057,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr AggregateDescription aggregate; aggregate.column_name = node->getColumnName(); - /// Агрегатные функции уникализируются. + /// Make unique aggregate functions. for (size_t i = 0; i < aggregate_descriptions.size(); ++i) if (aggregate_descriptions[i].column_name == aggregate.column_name) return; @@ -2068,7 +2068,7 @@ void ExpressionAnalyzer::getAggregates(const ASTPtr & ast, ExpressionActionsPtr for (size_t i = 0; i < arguments.size(); ++i) { - /// Внутри агрегатных функций не может быть других агрегатных функций. + /// There can not be other aggregate functions within the aggregate functions. assertNoAggregates(arguments[i], "inside another aggregate function"); getRootActions(arguments[i], true, false, actions); @@ -2158,7 +2158,7 @@ void ExpressionAnalyzer::addMultipleArrayJoinAction(ExpressionActionsPtr & actio if (result_source.first != result_source.second) actions->add(ExpressionAction::copyColumn(result_source.second, result_source.first)); - /// Сделать ARRAY JOIN (заменить массивы на их внутренности) для столбцов в этими новыми именами. + /// Make ARRAY JOIN (replace arrays with their insides) for the columns in these new names. result_columns.insert(result_source.first); } @@ -2209,13 +2209,13 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty if (join_params.using_expression_list) getRootActions(join_params.using_expression_list, only_types, false, step.actions); - /// Не поддерживается два JOIN-а с одинаковым подзапросом, но разными USING-ами. + /// Two JOINs are not supported with the same subquery, but different USINGs. String join_id = join_element.getTreeID(); SubqueryForSet & subquery_for_set = subqueries_for_sets[join_id]; - /// Особый случай - если справа JOIN указано имя таблицы, при чём, таблица имеет тип Join (заранее подготовленное отображение). - /// TODO В этом синтаксисе не поддерживается указание имени БД. + /// Special case - if table name is specified on the right of JOIN, then the table has the type Join (the previously prepared mapping). + /// TODO This syntax does not support specifying a database name. if (table_to_join.database_and_table_name) { StoragePtr table = context.tryGetTable("", static_cast(*table_to_join.database_and_table_name).name); @@ -2227,7 +2227,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty if (storage_join) { storage_join->assertCompatible(join_params.kind, join_params.strictness); - /// TODO Проверять набор ключей. + /// TODO Check the set of keys. JoinPtr & join = storage_join->getJoin(); subquery_for_set.join = join; @@ -2245,10 +2245,10 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty for (const auto & name_type : columns_added_by_join) required_joined_columns.push_back(name_type.name); - /** Для GLOBAL JOIN-ов (в случае, например, push-метода выполнения GLOBAL подзапросов) происходит следующее: - * - в функции addExternalStorage подзапрос JOIN (SELECT ...) заменяется на JOIN _data1, - * в объекте subquery_for_set выставляется этот подзапрос в качестве source и временная таблица _data1 в качестве table. - * - в этой функции видно выражение JOIN _data1. + /** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs + * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, + * in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`. + * - this function shows the expression JOIN _data1. */ if (!subquery_for_set.source) { @@ -2263,7 +2263,7 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty subquery_for_set.source_sample = interpreter->getSampleBlock(); } - /// TODO Это не нужно выставлять, когда JOIN нужен только на удалённых серверах. + /// TODO You do not need to set this up when JOIN is only needed on remote servers. subquery_for_set.join = join; subquery_for_set.join->setSampleBlock(subquery_for_set.source_sample); } @@ -2475,7 +2475,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result) } else { - /// Не будем удалять исходные столбцы. + /// We will not delete the original columns. for (const auto & column_name_type : columns) result_names.push_back(column_name_type.name); } @@ -2505,9 +2505,9 @@ void ExpressionAnalyzer::getAggregateInfo(Names & key_names, AggregateDescriptio void ExpressionAnalyzer::collectUsedColumns() { - /** Вычислим, какие столбцы требуются для выполнения выражения. - * Затем, удалим все остальные столбцы из списка доступных столбцов. - * После выполнения, columns будет содержать только список столбцов, нужных для чтения из таблицы. + /** Calculate which columns are required to execute the expression. + * Then, delete all other columns from the list of available columns. + * After execution, columns will only contain the list of columns needed to read from the table. */ NameSet required; @@ -2518,15 +2518,15 @@ void ExpressionAnalyzer::collectUsedColumns() ASTs & expressions = select_query->array_join_expression_list()->children; for (size_t i = 0; i < expressions.size(); ++i) { - /// Игнорируем идентификаторы верхнего уровня из секции ARRAY JOIN. - /// Их потом добавим отдельно. + /// Ignore the top-level identifiers from the ARRAY JOIN section. + /// Then add them separately. if (typeid_cast(expressions[i].get())) { ignored.insert(expressions[i]->getColumnName()); } else { - /// Для выражений в ARRAY JOIN ничего игнорировать не нужно. + /// Nothing needs to be ignored for expressions in ARRAY JOIN. NameSet empty; getRequiredColumnsImpl(expressions[i], required, empty, empty, empty); } @@ -2535,8 +2535,8 @@ void ExpressionAnalyzer::collectUsedColumns() } } - /** Также нужно не учитывать идентификаторы столбцов, получающихся путём JOIN-а. - * (Не считать, что они требуются для чтения из "левой" таблицы). + /** You also need to ignore the identifiers of the columns that are obtained by JOIN. + * (Do not assume that they are required for reading from the "left" table). */ NameSet available_joined_columns; collectJoinedColumns(available_joined_columns, columns_added_by_join); @@ -2556,7 +2556,7 @@ void ExpressionAnalyzer::collectUsedColumns() std::cerr << "JOINed column (required, not key): " << name_type.name << std::endl; std::cerr << std::endl;*/ - /// Вставляем в список требуемых столбцов столбцы, нужные для вычисления ARRAY JOIN. + /// Insert the columns required for the ARRAY JOIN calculation into the required columns list. NameSet array_join_sources; for (const auto & result_source : array_join_result_to_source) array_join_sources.insert(result_source.second); @@ -2565,7 +2565,7 @@ void ExpressionAnalyzer::collectUsedColumns() if (array_join_sources.count(column_name_type.name)) required.insert(column_name_type.name); - /// Нужно прочитать хоть один столбец, чтобы узнать количество строк. + /// You need to read at least one column to find the number of rows. if (required.empty()) required.insert(ExpressionActions::getSmallestColumn(columns)); @@ -2581,8 +2581,8 @@ void ExpressionAnalyzer::collectUsedColumns() ++it; } - /// Возможно, среди неизвестных столбцов есть виртуальные. Удаляем их из списка неизвестных и добавляем - /// в columns list, чтобы при дальнейшей обработке запроса они воспринимались как настоящие. + /// Perhaps, there are virtual columns among the unknown columns. Remove them from the list of unknown and add + /// in columns list, so that when further processing the request they are perceived as real. if (storage) { for (auto it = unknown_required_columns.begin(); it != unknown_required_columns.end();) @@ -2644,7 +2644,7 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd { const auto & col = nested_result_sample.safeGetByPosition(i); if (join_key_names_right.end() == std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name) - && !joined_columns.count(col.name)) /// Дублирующиеся столбцы в подзапросе для JOIN-а не имеют смысла. + && !joined_columns.count(col.name)) /// Duplicate columns in the subquery for JOIN do not make sense. { joined_columns.insert(col.name); joined_columns_name_type.emplace_back(col.name, col.type); @@ -2679,13 +2679,13 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(ASTPtr ast, NameSet & required_columns, NameSet & ignored_names, const NameSet & available_joined_columns, NameSet & required_joined_columns) { - /** Найдём все идентификаторы в запросе. - * Будем искать их рекурсивно, обходя в глубину AST. - * При этом: - * - для лямбда функций не будем брать формальные параметры; - * - не опускаемся в подзапросы (там свои идентификаторы); - * - некоторое исключение для секции ARRAY JOIN (в ней идентификаторы немного другие); - * - идентификаторы, доступные из JOIN-а, кладём в required_joined_columns. + /** Find all the identifiers in the query. + * We will look for them recursively, bypassing by depth AST. + * In this case + * - for lambda functions we will not take formal parameters; + * - do not go into subqueries (there are their identifiers); + * - is some exception for the ARRAY JOIN section (it has a slightly different identifier); + * - identifiers available from JOIN, we put in required_joined_columns. */ if (ASTIdentifier * node = typeid_cast(ast.get())) @@ -2715,7 +2715,7 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(ASTPtr ast, if (!lambda_args_tuple || lambda_args_tuple->name != "tuple") throw Exception("First argument of lambda must be a tuple", ErrorCodes::TYPE_MISMATCH); - /// Не нужно добавлять формальные параметры лямбда-выражения в required_columns. + /// You do not need to add formal parameters of the lambda expression in required_columns. Names added_ignored; for (auto & child : lambda_args_tuple->arguments->children) { @@ -2741,17 +2741,17 @@ void ExpressionAnalyzer::getRequiredColumnsImpl(ASTPtr ast, return; } - /// Особая функция indexHint. Всё, что внутри неё не вычисляется - /// (а используется только для анализа индекса, см. PKCondition). + /// A special function `indexHint`. Everything that is inside it is not calculated + /// (and is used only for index analysis, see PKCondition). if (node->name == "indexHint") return; } - /// Рекурсивный обход выражения. + /// Recursively traverses an expression. for (auto & child : ast->children) { - /** Не пойдем в секцию ARRAY JOIN, потому что там нужно смотреть на имена не-ARRAY-JOIN-енных столбцов. - * Туда collectUsedColumns отправит нас отдельно. + /** We will not go to the ARRAY JOIN section, because we need to look at the names of non-ARRAY-JOIN columns. + * There, `collectUsedColumns` will send us separately. */ if (!typeid_cast(child.get()) && !typeid_cast(child.get())) diff --git a/dbms/src/Interpreters/ExternalDictionaries.cpp b/dbms/src/Interpreters/ExternalDictionaries.cpp index 1e57b5047c3..ffef6af9e64 100644 --- a/dbms/src/Interpreters/ExternalDictionaries.cpp +++ b/dbms/src/Interpreters/ExternalDictionaries.cpp @@ -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 lock{dictionaries_mutex}; diff --git a/dbms/src/Interpreters/InterpreterCheckQuery.cpp b/dbms/src/Interpreters/InterpreterCheckQuery.cpp index b76f3ac3487..cba415a55eb 100644 --- a/dbms/src/Interpreters/InterpreterCheckQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCheckQuery.cpp @@ -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(&*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(); ColumnPtr host_name_column = std::make_shared(); @@ -216,7 +216,7 @@ BlockIO InterpreterCheckQuery::execute() ColumnPtr structure_class_column = std::make_shared(); ColumnPtr structure_column = std::make_shared(); - /// Это значение равно 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) diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index 0ded661b14d..de8477caffc 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -151,7 +151,7 @@ void InterpreterCreateQuery::createDatabase(ASTCreateQuery & create) using ColumnsAndDefaults = std::pair; -/// 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 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(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( std::make_shared( diff --git a/dbms/src/Interpreters/InterpreterDropQuery.cpp b/dbms/src/Interpreters/InterpreterDropQuery.cpp index d75196c3eca..4674beaa9e9 100644 --- a/dbms/src/Interpreters/InterpreterDropQuery.cpp +++ b/dbms/src/Interpreters/InterpreterDropQuery.cpp @@ -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); diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index 3d27628b37a..c78962ba9e3 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -84,7 +84,7 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & } else if (typeid_cast(query.get())) { - /// readonly проверяется внутри InterpreterSetQuery + /// readonly is checked inside InterpreterSetQuery return std::make_unique(query, context); } else if (typeid_cast(query.get())) diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 0ee09010822..295cd3b57b8 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -43,7 +43,7 @@ StoragePtr InterpreterInsertQuery::getTable() { ASTInsertQuery & query = typeid_cast(*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(*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(table->getColumnsList()); - /// Создаем конвейер из нескольких стримов, в которые будем писать данные. + /// We create a pipeline of several streams, into which we will write data. BlockOutputStreamPtr out; out = std::make_shared(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; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 0db0fb29215..f13dedc9eea 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -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(query_table.get())) { - /// Получить табличную функцию + /// Get the table function TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast(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(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(*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(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(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 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(); 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(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(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( @@ -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(streams[0], params, final, original_max_threads); } else @@ -1072,7 +1072,7 @@ void InterpreterSelectQuery::executeOrder() { auto sorting_stream = std::make_shared(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( 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(stream); }); - /// Сливаем сортированные источники в один сортированный источник. + /// Merge the sorted sources into one sorted source. streams[0] = std::make_shared(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>(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(); diff --git a/dbms/src/Interpreters/InterpreterSetQuery.cpp b/dbms/src/Interpreters/InterpreterSetQuery.cpp index fb37ce49485..ef7a2cc632f 100644 --- a/dbms/src/Interpreters/InterpreterSetQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSetQuery.cpp @@ -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); diff --git a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp index 31566b5e27d..e804437f20d 100644 --- a/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp +++ b/dbms/src/Interpreters/InterpreterShowTablesQuery.cpp @@ -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); diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index b491d8b055f..25b799625ff 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -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 { using Type = JoinKeyG template <> struct KeyGetterForType { 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 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(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 filter; if ((kind == ASTTableJoin::Kind::Inner || kind == ASTTableJoin::Kind::Right) && strictness == ASTTableJoin::Strictness::Any) filter = std::make_unique(rows); - /// Используется при ALL ... JOIN + /// Used with ALL ... JOIN IColumn::Offset_t current_offset = 0; std::unique_ptr offsets_to_replicate; if (strictness == ASTTableJoin::Strictness::All) offsets_to_replicate = std::make_unique(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 }; -/// Поток из неприсоединённых ранее строк правой таблицы. +/// 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); diff --git a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp index da045d09448..a54412c5e24 100644 --- a/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp +++ b/dbms/src/Interpreters/LogicalExpressionsOptimizer.cpp @@ -117,7 +117,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains() auto expression_list = typeid_cast(&*(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(&*child); @@ -126,7 +126,7 @@ void LogicalExpressionsOptimizer::collectDisjunctiveEqualityChains() auto equals_expression_list = typeid_cast(&*(equals->children[0])); if ((equals_expression_list != nullptr) && (equals_expression_list->children.size() == 2)) { - /// Равенство expr = xN. + /// Equality expr = xN. auto literal = typeid_cast(&*(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(&*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(); 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(&*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(); 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 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))) diff --git a/dbms/src/Interpreters/ProcessList.cpp b/dbms/src/Interpreters/ProcessList.cpp index ac20b03b48b..b94bfad98db 100644 --- a/dbms/src/Interpreters/ProcessList.cpp +++ b/dbms/src/Interpreters/ProcessList.cpp @@ -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 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 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); diff --git a/dbms/src/Interpreters/QueryLog.cpp b/dbms/src/Interpreters/QueryLog.cpp index 7a6c3075975..8e2d2f0641f 100644 --- a/dbms/src/Interpreters/QueryLog.cpp +++ b/dbms/src/Interpreters/QueryLog.cpp @@ -80,7 +80,7 @@ static std::array 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'; diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp index 2ff4f1dd4c8..27298a5c2fe 100644 --- a/dbms/src/Interpreters/Quota.cpp +++ b/dbms/src/Interpreters/Quota.cpp @@ -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); } diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index f08defc63db..8445050a031 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -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}; diff --git a/dbms/src/Interpreters/SetVariants.cpp b/dbms/src/Interpreters/SetVariants.cpp index 3f960620958..fd323c9ac53 100644 --- a/dbms/src/Interpreters/SetVariants.cpp +++ b/dbms/src/Interpreters/SetVariants.cpp @@ -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) diff --git a/dbms/src/Interpreters/Settings.cpp b/dbms/src/Interpreters/Settings.cpp index bce3fd16935..34343e68963 100644 --- a/dbms/src/Interpreters/Settings.cpp +++ b/dbms/src/Interpreters/Settings.cpp @@ -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 diff --git a/dbms/src/Interpreters/convertFieldToType.cpp b/dbms/src/Interpreters/convertFieldToType.cpp index 05610c7c310..3936af337ee 100644 --- a/dbms/src/Interpreters/convertFieldToType.cpp +++ b/dbms/src/Interpreters/convertFieldToType.cpp @@ -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 diff --git a/dbms/src/Interpreters/evaluateConstantExpression.cpp b/dbms/src/Interpreters/evaluateConstantExpression.cpp index 7c21ac20009..2444597799b 100644 --- a/dbms/src/Interpreters/evaluateConstantExpression.cpp +++ b/dbms/src/Interpreters/evaluateConstantExpression.cpp @@ -26,7 +26,7 @@ std::pair> evaluateConstantExpression(std::sha ExpressionActionsPtr expr_for_constant_folding = ExpressionAnalyzer( node, context, nullptr, NamesAndTypesList{{ "_dummy", std::make_shared() }}).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(1, 0), std::make_shared(), "_dummy" }}; expr_for_constant_folding->execute(block_with_constants); diff --git a/dbms/src/Interpreters/sortBlock.cpp b/dbms/src/Interpreters/sortBlock.cpp index 7e7e5973ae0..0287ece011d 100644 --- a/dbms/src/Interpreters/sortBlock.cpp +++ b/dbms/src/Interpreters/sortBlock.cpp @@ -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) { diff --git a/dbms/src/Interpreters/tests/hash_map.cpp b/dbms/src/Interpreters/tests/hash_map.cpp index e70eba67467..b2ddbaa7e95 100644 --- a/dbms/src/Interpreters/tests/hash_map.cpp +++ b/dbms/src/Interpreters/tests/hash_map.cpp @@ -20,32 +20,32 @@ #include -/** Тест проверяет скорость работы хэш-таблиц, имитируя их использование для агрегации. - * Первым аргументом указывается количество элементов, которое будет вставлено. - * Вторым аргументом может быть указано число от 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 diff --git a/dbms/src/Interpreters/tests/hash_map2.cpp b/dbms/src/Interpreters/tests/hash_map2.cpp index 4973c8c9427..02b96a9d6dc 100644 --- a/dbms/src/Interpreters/tests/hash_map2.cpp +++ b/dbms/src/Interpreters/tests/hash_map2.cpp @@ -40,15 +40,15 @@ struct CellWithoutZeroWithSavedHash : public HashMapCell { - /// Состояние этой структуры достаточно, чтобы получить размер буфера хэш-таблицы. + /// 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; - /// Из-за 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, Grower>; Map map; diff --git a/dbms/src/Interpreters/tests/hash_map_string.cpp b/dbms/src/Interpreters/tests/hash_map_string.cpp index bca9b97d3e0..d5058677b04 100644 --- a/dbms/src/Interpreters/tests/hash_map_string.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string.cpp @@ -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; - /// Сохранение хэша ускоряет ресайзы примерно в 2 раза, и общую производительность - на 6-8%. + /// Saving the hash accelerates the resize by about 2 times, and the overall performance by 6-8%. using Map = HashMapWithSavedHash, Grower>; Map map; diff --git a/dbms/src/Interpreters/tests/hash_map_string_2.cpp b/dbms/src/Interpreters/tests/hash_map_string_2.cpp index 5d2c6622893..d6dbd9f2d1f 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_2.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_2.cpp @@ -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(p1)), _mm_loadu_ps(reinterpret_cast(p2)))); } diff --git a/dbms/src/Interpreters/tests/hash_map_string_3.cpp b/dbms/src/Interpreters/tests/hash_map_string_3.cpp index 3e8252bb480..51262262130 100644 --- a/dbms/src/Interpreters/tests/hash_map_string_3.cpp +++ b/dbms/src/Interpreters/tests/hash_map_string_3.cpp @@ -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 diff --git a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp index 9a6b0a8c2ae..a786f061a94 100644 --- a/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp +++ b/dbms/src/Interpreters/tests/in_join_subqueries_preprocessor.cpp @@ -26,7 +26,7 @@ namespace DB } -/// Упрощённый вариант класса StorageDistributed. +/// Simplified version of the StorageDistributed class. class StorageDistributedFake : private ext::shared_ptr_helper, public DB::IStorage { friend class ext::shared_ptr_helper; @@ -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); diff --git a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp index fb339ba05fc..419d4f52d53 100644 --- a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp +++ b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp @@ -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); diff --git a/dbms/src/Interpreters/tests/select_query.cpp b/dbms/src/Interpreters/tests/select_query.cpp index 3a87eba6e49..24291682c14 100644 --- a/dbms/src/Interpreters/tests/select_query.cpp +++ b/dbms/src/Interpreters/tests/select_query.cpp @@ -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; diff --git a/dbms/src/Interpreters/tests/users.cpp b/dbms/src/Interpreters/tests/users.cpp index ad22b4bc2b0..4a3f482d01b 100644 --- a/dbms/src/Interpreters/tests/users.cpp +++ b/dbms/src/Interpreters/tests/users.cpp @@ -35,7 +35,7 @@ struct TestDescriptor using TestSet = std::vector; -/// Описание тестов. +/// Tests description. TestSet test_set = { diff --git a/dbms/src/Parsers/ASTAlterQuery.cpp b/dbms/src/Parsers/ASTAlterQuery.cpp index 0788243e9e4..8f19a0db5f1 100644 --- a/dbms/src/Parsers/ASTAlterQuery.cpp +++ b/dbms/src/Parsers/ASTAlterQuery.cpp @@ -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); diff --git a/dbms/src/Parsers/ASTFunction.cpp b/dbms/src/Parsers/ASTFunction.cpp index a84f7aad735..416eccbcb0b 100644 --- a/dbms/src/Parsers/ASTFunction.cpp +++ b/dbms/src/Parsers/ASTFunction.cpp @@ -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(&*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) { diff --git a/dbms/src/Parsers/ASTIdentifier.cpp b/dbms/src/Parsers/ASTIdentifier.cpp index a73d9aefc46..ac1e33f06b3 100644 --- a/dbms/src/Parsers/ASTIdentifier.cpp +++ b/dbms/src/Parsers/ASTIdentifier.cpp @@ -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) { diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index dd9ae8ca3b6..85699a7ef42 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -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(); 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; Mapping mapping(asts.size()); - /// На какой позиции в SELECT-выражении находится соответствующий столбец из column_names. + /// On which position in the SELECT expression is the corresponding column from `column_names`. std::vector 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(current.get())->prev_union_all = nullptr; ASTPtr next = static_cast(current.get())->next_union_all; @@ -187,15 +187,15 @@ std::shared_ptr 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(*next_union_all); next_ast.formatImpl(s, state, frame); diff --git a/dbms/src/Parsers/ASTWithAlias.cpp b/dbms/src/Parsers/ASTWithAlias.cpp index 2dd6ce70c96..fae4cf6b664 100644 --- a/dbms/src/Parsers/ASTWithAlias.cpp +++ b/dbms/src/Parsers/ASTWithAlias.cpp @@ -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 <<'('; diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 47752d17db6..9159599ee8f 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -84,7 +84,7 @@ bool ParserParenthesisExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, P ASTExpressionList & expr_list = typeid_cast(*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(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(*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::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(*node.get()).name; @@ -772,24 +772,24 @@ bool ParserWithOptionalAliasImpl::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) diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index 90fc99070a1..a595802e00c 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -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(); - /// аргументы функции + /// function arguments auto exp_list = std::make_shared(); 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(); auto args_and = std::make_shared(); @@ -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(); - /// аргументы функции + /// function arguments auto exp_list = std::make_shared(); 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(); - /// аргументы функции + /// function arguments auto exp_list = std::make_shared(); 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 == '-') { diff --git a/dbms/src/Parsers/IAST.cpp b/dbms/src/Parsers/IAST.cpp index 09fb80e6ff2..a096f6883c1 100644 --- a/dbms/src/Parsers/IAST.cpp +++ b/dbms/src/Parsers/IAST.cpp @@ -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'); diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index 53b1fc2ef68..0f476af24aa 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -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(*storage).name == "View" || typeid_cast(*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); diff --git a/dbms/src/Parsers/ParserInsertQuery.cpp b/dbms/src/Parsers/ParserInsertQuery.cpp index 77cc3165842..6b116e4277f 100644 --- a/dbms/src/Parsers/ParserInsertQuery.cpp +++ b/dbms/src/Parsers/ParserInsertQuery.cpp @@ -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); diff --git a/dbms/src/Parsers/ParserRenameQuery.cpp b/dbms/src/Parsers/ParserRenameQuery.cpp index 7bdeb919818..c13794e81d7 100644 --- a/dbms/src/Parsers/ParserRenameQuery.cpp +++ b/dbms/src/Parsers/ParserRenameQuery.cpp @@ -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) { diff --git a/dbms/src/Parsers/ParserSampleRatio.cpp b/dbms/src/Parsers/ParserSampleRatio.cpp index caa8842c916..423dfa8b89c 100644 --- a/dbms/src/Parsers/ParserSampleRatio.cpp +++ b/dbms/src/Parsers/ParserSampleRatio.cpp @@ -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) diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index e41fcdfc51a..55311d3ed4e 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -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); diff --git a/dbms/src/Parsers/ParserSetQuery.cpp b/dbms/src/Parsers/ParserSetQuery.cpp index 9738ed1074a..cdc5876658e 100644 --- a/dbms/src/Parsers/ParserSetQuery.cpp +++ b/dbms/src/Parsers/ParserSetQuery.cpp @@ -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;