diff --git a/dbms/src/AggregateFunctions/AggregateFunctionQuantileTDigest.h b/dbms/src/AggregateFunctions/AggregateFunctionQuantileTDigest.h index b404dc08192..f00ea788249 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionQuantileTDigest.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionQuantileTDigest.h @@ -248,7 +248,7 @@ public: return summary.back().mean; } - /** Get multiple quantiles (`size` pieces). + /** Get multiple quantiles (`size` parts). * levels - an array of levels of the desired quantiles. They are in a random order. * levels_permutation - array-permutation levels. The i-th position will be the index of the i-th ascending level in the `levels` array. * result - the array where the results are added, in order of `levels`, diff --git a/dbms/src/Storages/AlterCommands.h b/dbms/src/Storages/AlterCommands.h index 84235459546..543106254e6 100644 --- a/dbms/src/Storages/AlterCommands.h +++ b/dbms/src/Storages/AlterCommands.h @@ -6,7 +6,7 @@ namespace DB { -/// Операция из запроса ALTER (кроме манипуляции с PART/PARTITION). Добавление столбцов типа Nested не развернуто в добавление отдельных столбцов. +/// Operation from the ALTER query (except for manipulation with PART/PARTITION). Adding Nested columns is not expanded to add individual columns. struct AlterCommand { enum Type @@ -24,19 +24,19 @@ struct AlterCommand /// For DROP COLUMN ... FROM PARTITION String partition_name; - /// Для ADD и MODIFY - новый тип столбца. + /// For ADD and MODIFY, a new column type. DataTypePtr data_type; ColumnDefaultType default_type{}; ASTPtr default_expression{}; - /// Для ADD - после какого столбца добавить новый. Если пустая строка, добавить в конец. Добавить в начало сейчас нельзя. + /// For ADD - after which column to add a new one. If an empty string, add to the end. To add to the beginning now it is impossible. String after_column; - /// Для MODIFY_PRIMARY_KEY + /// For MODIFY_PRIMARY_KEY ASTPtr primary_key; - /// одинаковыми считаются имена, если они совпадают целиком или name_without_dot совпадает с частью имени до точки + /// the names are the same if they match the whole name or name_without_dot matches the part of the name up to the dot static bool namesEqual(const String & name_without_dot, const DB::NameAndTypePair & name_type) { String name_with_dot = name_without_dot + "."; diff --git a/dbms/src/Storages/CompressionMethodSelector.h b/dbms/src/Storages/CompressionMethodSelector.h index bc59f169e4a..5113a662b16 100644 --- a/dbms/src/Storages/CompressionMethodSelector.h +++ b/dbms/src/Storages/CompressionMethodSelector.h @@ -15,19 +15,19 @@ namespace ErrorCodes } -/** Позволяет выбрать метод сжатия по указанным в конфигурационном файле условиям. - * Конфиг выглядит примерно так: +/** Allows you to select the compression method for the conditions specified in the configuration file. + * The config looks like this - + - - 10000000000 - 0.01 + + 10000000000 + 0.01 - + zstd @@ -73,7 +73,7 @@ private: std::vector elements; public: - CompressionMethodSelector() {} /// Всегда возвращает метод по-умолчанию. + CompressionMethodSelector() {} /// Always returns the default method. CompressionMethodSelector(Poco::Util::AbstractConfiguration & config, const std::string & config_prefix) { diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h index 6ff0c31f89e..f0a196b5850 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -10,14 +10,14 @@ namespace DB class StorageDistributed; -/** Запись асинхронная - данные сначала записываются на локальную файловую систему, а потом отправляются на удалённые серверы. - * Если Distributed таблица использует более одного шарда, то для того, чтобы поддерживалась запись, - * при создании таблицы должен быть указан дополнительный параметр у ENGINE - ключ шардирования. - * Ключ шардирования - произвольное выражение от столбцов. Например, rand() или UserID. - * При записи блок данных разбивается по остатку от деления ключа шардирования на суммарный вес шардов, - * и полученные блоки пишутся в сжатом Native формате в отдельные директории для отправки. - * Для каждого адреса назначения (каждой директории с данными для отправки), в StorageDistributed создаётся отдельный поток, - * который следит за директорией и отправляет данные. */ +/** The write is asynchronous - the data is first written to the local file system, and then sent to the remote servers. + * If the Distributed table uses more than one shard, then in order to support the write, + * when creating the table, an additional parameter must be specified for ENGINE - the sharding key. + * Sharding key is an arbitrary expression from the columns. For example, rand() or UserID. + * When writing, the data block is splitted by the remainder of the division of the sharding key by the total weight of the shards, + * and the resulting blocks are written in a compressed Native format in separate directories for sending. + * For each destination address (each directory with data to send), a separate thread is created in StorageDistributed, + * which monitors the directory and sends data. */ class DistributedBlockOutputStream : public IBlockOutputStream { public: diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index a50805b2688..6aeff7d96b4 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -44,11 +44,11 @@ using WeightedZooKeeperPaths = std::vector; -/** Не дает изменять описание таблицы (в том числе переименовывать и удалять таблицу). - * Если в течение какой-то операции структура таблицы должна оставаться неизменной, нужно держать такой лок на все ее время. - * Например, нужно держать такой лок на время всего запроса SELECT или INSERT и на все время слияния набора кусков - * (но между выбором кусков для слияния и их слиянием структура таблицы может измениться). - * NOTE: Это лок на "чтение" описания таблицы. Чтобы изменить описание таблицы, нужно взять TableStructureWriteLock. +/** Does not allow changing the table description (including rename and delete the table). + * If during any operation the table structure should remain unchanged, you need to hold such a lock for all of its time. + * For example, you need to hold such a lock for the duration of the entire SELECT or INSERT query and for the whole time the merge of the set of parts + * (but between the selection of parts for the merge and their merging, the table structure can change). + * NOTE: This is a lock to "read" the table's description. To change the table description, you need to take the TableStructureWriteLock. */ class TableStructureReadLock { @@ -56,7 +56,7 @@ private: friend class IStorage; StoragePtr storage; - /// Порядок важен. + /// Order is important. std::experimental::optional data_lock; std::experimental::optional structure_lock; @@ -72,39 +72,39 @@ using TableDataWriteLockPtr = std::unique_ptr; using TableFullWriteLockPtr = std::pair; -/** Хранилище. Отвечает за: - * - хранение данных таблицы; - * - определение, в каком файле (или не файле) хранятся данные; - * - поиск данных и обновление данных; - * - структура хранения данных (сжатие, etc.) - * - конкуррентный доступ к данным (блокировки, etc.) +/** Storage. Responsible for + * - storage of the table data; + * - the definition in which file (or not the file) the data is stored; + * - search for data and update data; + * - data storage structure (compression, etc.) + * - concurrent access to data (locks, etc.) */ class IStorage : public std::enable_shared_from_this, private boost::noncopyable, public ITableDeclaration { public: - /// Основное имя типа таблицы (например, StorageMergeTree). + /// The main name of the table type (for example, StorageMergeTree). virtual std::string getName() const = 0; - /** Возвращает true, если хранилище получает данные с удалённого сервера или серверов. */ + /** Returns true if the store receives data from a remote server or servers. */ virtual bool isRemote() const { return false; } - /** Возвращает true, если хранилище поддерживает запросы с секцией SAMPLE. */ + /** Returns true if the storage supports queries with the SAMPLE section. */ virtual bool supportsSampling() const { return false; } - /** Возвращает true, если хранилище поддерживает запросы с секцией FINAL. */ + /** Returns true if the storage supports queries with the FINAL section. */ virtual bool supportsFinal() const { return false; } - /** Возвращает true, если хранилище поддерживает запросы с секцией PREWHERE. */ + /** Returns true if the storage supports queries with the PREWHERE section. */ virtual bool supportsPrewhere() const { return false; } - /** Возвращает true, если хранилище поддерживает несколько реплик. */ + /** Returns true if the storage supports multiple replicas. */ virtual bool supportsParallelReplicas() const { return false; } - /** Не дает изменять структуру или имя таблицы. - * Если в рамках этого лока будут изменены данные в таблице, нужно указать will_modify_data=true. - * Это возьмет дополнительный лок, не позволяющий начать ALTER MODIFY. + /** Does not allow you to change the structure or name of the table. + * If you change the data in the table, you will need to specify will_modify_data = true. + * This will take an extra lock that does not allow starting ALTER MODIFY. * - * WARNING: Вызывать методы из ITableDeclaration нужно под такой блокировкой. Без нее они не thread safe. + * WARNING: You need to call methods from ITableDeclaration under such a lock. Without it, they are not thread safe. * WARNING: To avoid deadlocks, this method must not be called under lock of Context. */ TableStructureReadLockPtr lockStructure(bool will_modify_data) @@ -115,20 +115,20 @@ public: return res; } - /** Не дает читать структуру таблицы. Берется для ALTER, RENAME и DROP. + /** Does not allow reading the table structure. It is taken for ALTER, RENAME and DROP. */ TableFullWriteLockPtr lockForAlter() { - /// Порядок вычисления важен. + /// The calculation order is important. auto data_lock = lockDataForAlter(); auto structure_lock = lockStructureForAlter(); return {std::move(data_lock), std::move(structure_lock)}; } - /** Не дает изменять данные в таблице. (Более того, не дает посмотреть на структуру таблицы с намерением изменить данные). - * Берется на время записи временных данных в ALTER MODIFY. - * Под этим локом можно брать lockStructureForAlter(), чтобы изменить структуру таблицы. + /** Does not allow changing the data in the table. (Moreover, does not give a look at the structure of the table with the intention to change the data). + * It is taken during write temporary data in ALTER MODIFY. + * Under this lock, you can take lockStructureForAlter() to change the structure of the table. */ TableDataWriteLockPtr lockDataForAlter() { @@ -147,24 +147,24 @@ public: } - /** Читать набор столбцов из таблицы. - * Принимает список столбцов, которых нужно прочитать, а также описание запроса, - * из которого может быть извлечена информация о том, каким способом извлекать данные - * (индексы, блокировки и т. п.) - * Возвращает поток с помощью которого можно последовательно читать данные - * или несколько потоков для параллельного чтения данных. - * Также в processed_stage записывается, до какой стадии запрос был обработан. - * (Обычно функция только читает столбцы из списка, но в других случаях, - * например, запрос может быть частично обработан на удалённом сервере.) + /** Read a set of columns from the table. + * Accepts a list of columns to read, as well as a description of the query, + * from which information can be extracted about how to retrieve data + * (indexes, locks, etc.) + * Returns a stream with which you can read data sequentially + * or multiple streams for parallel data reading. + * The into `processed_stage` info is also written to what stage the request was processed. + * (Normally, the function only reads the columns from the list, but in other cases, + * for example, the request can be partially processed on a remote server.) * - * settings - настройки на один запрос. - * Обычно Storage не заботится об этих настройках, так как они применяются в интерпретаторе. - * Но, например, при распределённой обработке запроса, настройки передаются на удалённый сервер. + * settings - settings for one query. + * Usually Storage does not care about these settings, since they are used in the interpreter. + * But, for example, for distributed query processing, the settings are passed to the remote server. * - * threads - рекомендация, сколько потоков возвращать, - * если хранилище может возвращать разное количество потоков. + * threads - a recommendation, how many threads to return, + * if the storage can return a different number of threads. * - * Гарантируется, что структура таблицы не изменится за время жизни возвращенных потоков (то есть не будет ALTER, RENAME и DROP). + * It is guaranteed that the structure of the table will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). */ virtual BlockInputStreams read( const Names & column_names, @@ -178,11 +178,11 @@ public: throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Пишет данные в таблицу. - * Принимает описание запроса, в котором может содержаться информация о методе записи данных. - * Возвращает объект, с помощью которого можно последовательно писать данные. + /** Writes the data to a table. + * Receives a description of the query, which can contain information about the data write method. + * Returns an object by which you can write data sequentially. * - * Гарантируется, что структура таблицы не изменится за время жизни возвращенных потоков (то есть не будет ALTER, RENAME и DROP). + * It is guaranteed that the table structure will not change over the lifetime of the returned streams (that is, there will not be ALTER, RENAME and DROP). */ virtual BlockOutputStreamPtr write( ASTPtr query, @@ -191,24 +191,24 @@ public: throw Exception("Method write is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Удалить данные таблицы. Вызывается перед удалением директории с данными. - * Если не требуется никаких действий, кроме удаления директории с данными, этот метод можно оставить пустым. + /** Delete the table data. Called before deleting the directory with the data. + * If you do not need any action other than deleting the directory with data, you can leave this method blank. */ virtual void drop() {} - /** Переименовать таблицу. - * Переименование имени в файле с метаданными, имени в списке таблиц в оперативке, осуществляется отдельно. - * В этой функции нужно переименовать директорию с данными, если она есть. - * Вызывается при заблокированной на запись структуре таблицы. + /** Rename the table. + * Renaming a name in a file with metadata, the name in the list of tables in the RAM, is done separately. + * In this function, you need to rename the directory with the data, if any. + * Called when the table structure is locked for write. */ virtual void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) { throw Exception("Method rename is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** ALTER таблицы в виде изменения столбцов, не затрагивающий изменение Storage или его параметров. - * Этот метод должен полностью выполнить запрос ALTER, самостоятельно заботясь о блокировках. - * Для обновления метаданных таблицы на диске этот метод должен вызвать InterpreterAlterQuery::updateMetadata. + /** ALTER tables in the form of column changes that do not affect the change to Storage or its parameters. + * This method must fully execute the ALTER query, taking care of the locks itself. + * To update the table metadata on disk, this method should call InterpreterAlterQuery::updateMetadata. */ virtual void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) { @@ -221,35 +221,35 @@ public: throw Exception("Method dropColumnFromPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Выполнить запрос (DROP|DETACH) PARTITION. + /** Run the query (DROP|DETACH) PARTITION. */ virtual void dropPartition(ASTPtr query, const Field & partition, bool detach, bool unreplicated, const Settings & settings) { throw Exception("Method dropPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Выполнить запрос ATTACH [UNREPLICATED] (PART|PARTITION). + /** Run the ATTACH request [UNREPLICATED] (PART|PARTITION). */ virtual void attachPartition(ASTPtr query, const Field & partition, bool unreplicated, bool part, const Settings & settings) { throw Exception("Method attachPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Выполнить запрос FETCH PARTITION. + /** Run the FETCH PARTITION query. */ virtual void fetchPartition(const Field & partition, const String & from, const Settings & settings) { throw Exception("Method fetchPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Выполнить запрос FREEZE PARTITION. То есть, создать локальный бэкап (снэпшот) данных с помощью функции localBackup (см. localBackup.h) + /** Run the FREEZE PARTITION request. That is, create a local backup (snapshot) of data using the `localBackup` function (see localBackup.h) */ virtual void freezePartition(const Field & partition, const String & with_name, const Settings & settings) { throw Exception("Method freezePartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Выполнить запрос RESHARD PARTITION. + /** Run the RESHARD PARTITION query. */ virtual void reshardPartitions(ASTPtr query, const String & database_name, const Field & first_partition, const Field & last_partition, @@ -260,18 +260,18 @@ public: throw Exception("Method reshardPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Выполнить какую-либо фоновую работу. Например, объединение кусков в таблице типа MergeTree. - * Возвращает - была ли выполнена какая-либо работа. + /** Perform any background work. For example, combining parts in a MergeTree type table. + * Returns whether any work has been done. */ virtual bool optimize(const String & partition, bool final, const Settings & settings) { throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } - /** Если при уничтожении объекта надо сделать какую-то сложную работу - сделать её заранее. - * Например, если таблица содержит какие-нибудь потоки для фоновой работы - попросить их завершиться и дождаться завершения. - * По-умолчанию - ничего не делать. - * Может вызываться одновременно из разных потоков, даже после вызова drop(). + /** If you have to do some complicated work when destroying an object - do it in advance. + * For example, if the table contains any threads for background work - ask them to complete and wait for completion. + * By default, do nothing. + * Can be called simultaneously from different threads, even after a call to drop(). */ virtual void shutdown() {} @@ -295,25 +295,25 @@ protected: private: friend class TableStructureReadLock; - /// Брать следующие два лока всегда нужно в этом порядке. + /// You always need to take the next two locks in this order. - /** Берется на чтение на все время запроса INSERT и на все время слияния кусков (для MergeTree). - * Берется на запись на все время ALTER MODIFY. + /** It is taken for read for the entire INSERT query and the entire merge of the parts (for MergeTree). + * It is taken for write for the entire time ALTER MODIFY. * - * Формально: - * Ввзятие на запись гарантирует, что: - * 1) данные в таблице не изменится, пока лок жив, - * 2) все изменения данных после отпускания лока будут основаны на структуре таблицы на момент после отпускания лока. - * Нужно брать на чтение на все время операции, изменяющей данные. + * Formally: + * Taking a write lock ensures that: + * 1) the data in the table will not change while the lock is alive, + * 2) all changes to the data after releasing the lock will be based on the structure of the table at the time after the lock was released. + * You need to take for read for the entire time of the operation that changes the data. */ mutable Poco::RWLock data_lock; - /** Лок для множества столбцов и пути к таблице. Берется на запись в RENAME, ALTER (для ALTER MODIFY ненадолго) и DROP. - * Берется на чтение на все время SELECT, INSERT и слияния кусков (для MergeTree). + /** Lock for multiple columns and path to table. It is taken for write at RENAME, ALTER (for ALTER MODIFY for a while) and DROP. + * It is taken for read for the whole time of SELECT, INSERT and merge parts (for MergeTree). * - * Взятие этого лока на запись - строго более "сильная" операция, чем взятие parts_writing_lock на запись. - * То есть, если этот лок взят на запись, о parts_writing_lock можно не заботиться. - * parts_writing_lock нужен только для случаев, когда не хочется брать table_structure_lock надолго (ALTER MODIFY). + * Taking this lock for writing is a strictly "stronger" operation than taking parts_writing_lock for write record. + * That is, if this lock is taken for write, you should not worry about `parts_writing_lock`. + * parts_writing_lock is only needed for cases when you do not want to take `table_structure_lock` for long operations (ALTER MODIFY). */ mutable Poco::RWLock structure_lock; }; @@ -321,7 +321,7 @@ private: using StorageVector = std::vector; using StorageList = std::list; -/// имя таблицы -> таблица +/// table name -> table using Tables = std::map; } diff --git a/dbms/src/Storages/ITableDeclaration.h b/dbms/src/Storages/ITableDeclaration.h index e8daf195cf2..4861a36205f 100644 --- a/dbms/src/Storages/ITableDeclaration.h +++ b/dbms/src/Storages/ITableDeclaration.h @@ -15,67 +15,67 @@ namespace DB class Context; -/** Описание таблицы. - * Не thread safe. См. IStorage::lockStructure(). +/** Description of the table. + * Do not thread safe. See IStorage::lockStructure (). */ class ITableDeclaration { public: - /** Имя таблицы. + /** The name of the table. */ virtual std::string getTableName() const = 0; - /** Получить список имён и типов столбцов таблицы, только невиртуальные. + /** Get a list of names and table column types, only non-virtual. */ NamesAndTypesList getColumnsList() const; const NamesAndTypesList & getColumnsListNonMaterialized() const { return getColumnsListImpl(); } - /** Получить список имён столбцов таблицы, только невиртуальные. + /** Get a list of column table names, only non-virtual. */ virtual Names getColumnNamesList() const; - /** Получить описание реального (невиртуального) столбца по его имени. + /** Get a description of the real (non-virtual) column by its name. */ virtual NameAndTypePair getRealColumn(const String & column_name) const; - /** Присутствует ли реальный (невиртуальный) столбец с таким именем. + /** Is there a real (non-virtual) column with that name. */ virtual bool hasRealColumn(const String & column_name) const; NameAndTypePair getMaterializedColumn(const String & column_name) const; bool hasMaterializedColumn(const String & column_name) const; - /** Получить описание любого столбца по его имени. + /** Get a description of any column by its name. */ virtual NameAndTypePair getColumn(const String & column_name) const; - /** Присутствует ли столбец с таким именем. + /** Is there a column with that name. */ virtual bool hasColumn(const String & column_name) const; const DataTypePtr getDataTypeByName(const String & column_name) const; - /** То же самое, но в виде блока-образца. + /** The same, but in the form of a block-sample. */ Block getSampleBlock() const; Block getSampleBlockNonMaterialized() const; - /** Проверить, что все запрошенные имена есть в таблице и заданы корректно. - * (список имён не пустой и имена не повторяются) + /** Verify that all the requested names are in the table and are set correctly. + * (the list of names is not empty and the names do not repeat) */ void check(const Names & column_names) const; - /** Проверить, что все запрошенные имена есть в таблице и имеют правильные типы. + /** Check that all the requested names are in the table and have the correct types. */ void check(const NamesAndTypesList & columns) const; - /** Проверить, что все имена из пересечения names и columns есть в таблице и имеют одинаковые типы. + /** Check that all names from the intersection of `names` and `columns` are in the table and have the same types. */ void check(const NamesAndTypesList & columns, const Names & column_names) const; - /** Проверить, что блок с данными для записи содержит все столбцы таблицы с правильными типами, - * содержит только столбцы таблицы, и все столбцы различны. - * Если need_all, еще проверяет, что все столбцы таблицы есть в блоке. + /** Check that the data block for the record contains all the columns of the table with the correct types, + * contains only the columns of the table, and all the columns are different. + * If need_all, still checks that all the columns of the table are in the block. */ void check(const Block & block, bool need_all = false) const; diff --git a/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h b/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h index dc2350123a7..bdb0defe8d8 100644 --- a/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h +++ b/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h @@ -13,11 +13,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -/** Примитив синхронизации. Работает следующим образом: - * При создании создает неэфемерную инкрементную ноду и помечает ее как заблокированную (LOCKED). - * unlock() разблокирует ее (UNLOCKED). - * При вызове деструктора или завершении сессии в ZooKeeper, переходит в состояние ABANDONED. - * (В том числе при падении программы). +/** The synchronization is primitive. Works as follows: + * Creates a non-ephemeral incremental node and marks it as locked (LOCKED). + * `unlock()` unlocks it (UNLOCKED). + * When the destructor is called or the session ends in ZooKeeper, it goes into the ABANDONED state. + * (Including when the program is halted). */ class AbandonableLockInZooKeeper : private boost::noncopyable { @@ -33,10 +33,10 @@ public: const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_) : zookeeper(zookeeper_), path_prefix(path_prefix_) { - /// Создадим вспомогательную эфемерную ноду. + /// Let's create an secondary ephemeral node. holder_path = zookeeper.create(temp_path + "/abandonable_lock-", "", zkutil::CreateMode::EphemeralSequential); - /// Запишем в основную ноду путь к вспомогательной. + /// Write the path to the secondary node in the main node. path = zookeeper.create(path_prefix, holder_path, zkutil::CreateMode::PersistentSequential); if (path.size() <= path_prefix.size()) @@ -56,7 +56,7 @@ public: return path; } - /// Распарсить число в конце пути. + /// Parse the number at the end of the path. UInt64 getNumber() const { return parse(path.c_str() + path_prefix.size(), path.size() - path_prefix.size()); @@ -69,7 +69,7 @@ public: holder_path = ""; } - /// Добавляет в список действия, эквивалентные unlock(). + /// Adds actions equivalent to `unlock()` to the list. void getUnlockOps(zkutil::Ops & ops) { ops.emplace_back(std::make_unique(path, -1)); @@ -84,7 +84,7 @@ public: try { zookeeper.tryRemoveEphemeralNodeWithRetries(holder_path); - zookeeper.trySet(path, ""); /// Это не обязательно. + zookeeper.trySet(path, ""); /// It's not necessary. } catch (...) { @@ -96,21 +96,21 @@ public: { String holder_path; - /// Если нет основной ноды, UNLOCKED. + /// If there is no main node, UNLOCKED. if (!zookeeper.tryGet(path, holder_path)) return UNLOCKED; - /// Если в основной ноде нет пути к вспомогательной, ABANDONED. + /// If there is no path to the secondary node in the main node, ABANDONED. if (holder_path.empty()) return ABANDONED; - /// Если вспомогательная нода жива, LOCKED. + /// If the secondary node is alive, LOCKED. if (zookeeper.exists(holder_path)) return LOCKED; - /// Если вспомогательной ноды нет, нужно еще раз проверить существование основной ноды, - /// потому что за это время могли успеть вызвать unlock(). - /// Заодно уберем оттуда путь к вспомогательной ноде. + /// If there is no secondary node, you need to test again the existence of the main node, + /// because during this time you might have time to call unlock(). + /// At the same time, we will remove the path to the secondary node from there. if (zookeeper.trySet(path, "") == ZOK) return ABANDONED; diff --git a/dbms/src/Storages/MergeTree/ActiveDataPartSet.h b/dbms/src/Storages/MergeTree/ActiveDataPartSet.h index 9be54d7ce78..58f89574fb6 100644 --- a/dbms/src/Storages/MergeTree/ActiveDataPartSet.h +++ b/dbms/src/Storages/MergeTree/ActiveDataPartSet.h @@ -8,10 +8,10 @@ namespace DB { -/** Поддерживает множество названий активных кусков данных. - * Повторяет часть функциональности MergeTreeData. - * TODO: обобщить с MergeTreeData. Можно этот класс оставить примерно как есть и использовать его из MergeTreeData. - * Тогда в MergeTreeData можно сделать map data_parts и all_data_parts. +/** Supports multiple names of active parts of data. + * Repeats part of the MergeTreeData functionality. + * TODO: generalize with MergeTreeData. It is possible to leave this class approximately as is and use it from MergeTreeData. + * Then in MergeTreeData you can make map data_parts and all_data_parts. */ class ActiveDataPartSet { @@ -45,10 +45,10 @@ public: return false; } - /// Содержит другой кусок (получен после объединения другого куска с каким-то ещё) + /// Contains another part (obtained after combining another part with some other) bool contains(const Part & rhs) const { - return month == rhs.month /// Куски за разные месяцы не объединяются + return month == rhs.month /// Parts for different months are not combined && left_date <= rhs.left_date && right_date >= rhs.right_date && left <= rhs.left @@ -59,19 +59,19 @@ public: void add(const String & name); - /// Если не найдено - возвращает пустую строку. + /// If not found, returns an empty string. String getContainingPart(const String & name) const; - Strings getParts() const; /// В порядке возрастания месяца и номера блока. + Strings getParts() const; /// In ascending order of the month and block number. size_t size() const; static String getPartName(DayNum_t left_date, DayNum_t right_date, Int64 left_id, Int64 right_id, UInt64 level); - /// Возвращает true если имя директории совпадает с форматом имени директории кусочков + /// Returns true if the directory name matches the format of the directory name of the parts static bool isPartDirectory(const String & dir_name, Poco::RegularExpression::MatchVec * out_matches = nullptr); - /// Кладет в DataPart данные из имени кусочка. + /// Put data in DataPart from the name of the part. static void parsePartName(const String & file_name, Part & part, const Poco::RegularExpression::MatchVec * matches = nullptr); static bool contains(const String & outer_part_name, const String & inner_part_name); @@ -82,7 +82,7 @@ private: mutable std::mutex mutex; Parts parts; - /// Не блокируют mutex. + /// Do not block mutex. void addImpl(const String & name); String getContainingPartImpl(const String & name) const; }; diff --git a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h index 35492882d1e..a65fb524c3b 100644 --- a/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h +++ b/dbms/src/Storages/MergeTree/BackgroundProcessingPool.h @@ -15,11 +15,11 @@ namespace DB { -/** Используя фиксированное количество потоков, выполнять произвольное количество задач в бесконечном цикле. - * При этом, одна задача может выполняться одновременно из разных потоков. - * Предназначена для задач, выполняющих постоянную фоновую работу (например, слияния). - * Задача - функция, возвращающая bool - сделала ли она какую-либо работу. - * Если не сделала, то в следующий раз будет выполнена позже. +/** Using a fixed number of threads, perform an arbitrary number of tasks in an infinite loop. + * In this case, one task can run simultaneously from different threads. + * Designed for tasks that perform continuous background work (for example, merge). + * `Task` is a function that returns a bool - did it do any work. + * If not, then the next time will be done later. */ class BackgroundProcessingPool { diff --git a/dbms/src/Storages/MergeTree/BoolMask.h b/dbms/src/Storages/MergeTree/BoolMask.h index 76da783404d..a7e66a917b0 100644 --- a/dbms/src/Storages/MergeTree/BoolMask.h +++ b/dbms/src/Storages/MergeTree/BoolMask.h @@ -1,6 +1,6 @@ #pragma once -/// Множество значений булевой переменной. То есть два булевых значения: может ли быть true, может ли быть false. +/// Multiple Boolean values. That is, two Boolean values: can it be true, can it be false. struct BoolMask { bool can_be_true; diff --git a/dbms/src/Storages/MergeTree/DataPartsExchange.h b/dbms/src/Storages/MergeTree/DataPartsExchange.h index ff2ede733dd..1bb1d906c38 100644 --- a/dbms/src/Storages/MergeTree/DataPartsExchange.h +++ b/dbms/src/Storages/MergeTree/DataPartsExchange.h @@ -12,7 +12,7 @@ namespace DB namespace DataPartsExchange { -/** Сервис для отправки кусков из таблицы *MergeTree. +/** Service for sending parts from the table *MergeTree. */ class Service final : public InterserverIOEndpoint { @@ -36,7 +36,7 @@ private: Logger * log; }; -/** Клиент для получения кусков из таблицы *MergeTree. +/** Client for getting the parts from the table *MergeTree. */ class Fetcher final { @@ -46,7 +46,7 @@ public: Fetcher(const Fetcher &) = delete; Fetcher & operator=(const Fetcher &) = delete; - /// Скачивает кусок в tmp_директорию. Если to_detached - скачивает в директорию detached. + /// Downloads a part to tmp_directory. If to_detached - downloads to the `detached` directory. MergeTreeData::MutableDataPartPtr fetchPart( const String & part_name, const String & replica_path, @@ -54,8 +54,8 @@ public: int port, bool to_detached = false); - /// Метод для перешардирования. Скачивает шардированный кусок - /// из заданного шарда в папку to_detached. + /// Method for resharding. Downloads a sharded part + /// from the specified shard to the `to_detached` folder. MergeTreeData::MutableDataPartPtr fetchShardedPart( const InterserverIOEndpointLocation & location, const String & part_name, @@ -74,7 +74,7 @@ private: private: MergeTreeData & data; - /// Нужно остановить передачу данных. + /// You need to stop the data transfer. std::atomic is_cancelled {false}; Logger * log; }; diff --git a/dbms/src/Storages/MergeTree/MarkRange.h b/dbms/src/Storages/MergeTree/MarkRange.h index e2fa1ff407c..3d1701d2c02 100644 --- a/dbms/src/Storages/MergeTree/MarkRange.h +++ b/dbms/src/Storages/MergeTree/MarkRange.h @@ -8,7 +8,7 @@ namespace DB { -/** Пара засечек, определяющая диапазон строк в куске. Именно, диапазон имеет вид [begin * index_granularity, end * index_granularity). +/** A pair of marks that defines the range of rows in a part. Specifically, the range has the form [begin * index_granularity, end * index_granularity). */ struct MarkRange { diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index e43b980956d..bb6fa5261a0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -13,11 +13,11 @@ using MergeTreeReadTaskPtr = std::unique_ptr; using MergeTreeBlockSizePredictorPtr = std::shared_ptr; -/** Если некоторых запрошенных столбцов нет в куске, - * то выясняем, какие столбцы может быть необходимо дополнительно прочитать, - * чтобы можно было вычислить DEFAULT выражение для этих столбцов. - * Добавляет их в columns. - */ +/** If some of the requested columns are not in the part, + * then find out which columns may need to be read further, + * so that you can calculate the DEFAULT expression for these columns. + * Adds them to the `columns`. + */ NameSet injectRequiredColumns(const MergeTreeData & storage, const MergeTreeData::DataPartPtr & part, Names & columns); diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 10c17eeeeee..78d48bdf7bd 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -400,7 +400,6 @@ public: SortDescription getSortDescription() const { return sort_descr; } /// Check that the part is not broken and calculate the checksums for it if they are not present. - /// Проверить, что кусок не сломан и посчитать для него чексуммы, если их нет. MutableDataPartPtr loadPartAndFixMetadata(const String & relative_path); /** Create local backup (snapshot) for parts with specified prefix. diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.h b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.h index 32abf3e9628..d610fb0f2b5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.h @@ -13,7 +13,7 @@ class MergeProgressCallback; struct ReshardingJob; -/** Умеет выбирать куски для слияния и сливать их. +/** Can select the parts to merge and merge them. */ class MergeTreeDataMerger { @@ -36,12 +36,12 @@ public: */ size_t getMaxPartsSizeForMerge(size_t pool_size, size_t pool_used); - /** Выбирает, какие куски слить. Использует кучу эвристик. + /** Selects which parts to merge. Uses a lot of heuristics. * - * can_merge - функция, определяющая, можно ли объединить пару соседних кусков. - * Эта функция должна координировать слияния со вставками и другими слияниями, обеспечивая, что: - * - Куски, между которыми еще может появиться новый кусок, нельзя сливать. См. METR-7001. - * - Кусок, который уже сливается с кем-то в одном месте, нельзя начать сливать в кем-то другим в другом месте. + * can_merge - a function that determines if it is possible to merge a pair of adjacent parts. + * This function must coordinate merge with inserts and other merges, ensuring that + * - Parts between which another part can still appear can not be merged. Refer to METR-7001. + * - A part that already merges with something in one place, you can not start to merge into something else in another place. */ bool selectPartsToMerge( MergeTreeData::DataPartsVector & what, @@ -50,8 +50,8 @@ public: size_t max_total_size_to_merge, const AllowedMergingPredicate & can_merge); - /** Выбрать для слияния все куски в заданной партиции, если возможно. - * final - выбирать для слияния даже единственный кусок - то есть, позволять мерджить один кусок "сам с собой". + /** Select all the parts in the specified partition for merge, if possible. + * final - choose to merge even a single part - that is, allow to measure one part "with itself". */ bool selectAllPartsToMergeWithinPartition( MergeTreeData::DataPartsVector & what, @@ -61,15 +61,15 @@ public: DayNum_t partition, bool final); - /** Сливает куски. - * Если reservation != nullptr, то и дело уменьшает размер зарезервированного места - * приблизительно пропорционально количеству уже выписанных данных. + /** Merge the parts. + * If `reservation != nullptr`, now and then reduces the size of the reserved space + * is approximately proportional to the amount of data already written. * - * Создаёт и возвращает временный кусок. - * Чтобы закончить мердж, вызовите функцию renameTemporaryMergedPart. + * Creates and returns a temporary part. + * To end the merge, call the function renameTemporaryMergedPart. * - * time_of_merge - время, когда мердж был назначен. - * Важно при использовании ReplicatedGraphiteMergeTree для обеспечения одинакового мерджа на репликах. + * time_of_merge - the time when the merge was assigned. + * Important when using ReplicatedGraphiteMergeTree to provide the same merge on replicas. */ MergeTreeData::MutableDataPartPtr mergePartsToTemporaryPart( MergeTreeData::DataPartsVector & parts, const String & merged_name, MergeListEntry & merge_entry, @@ -81,17 +81,17 @@ public: const String & merged_name, MergeTreeData::Transaction * out_transaction = nullptr); - /** Перешардирует заданную партицию. + /** Reshards the specified partition. */ MergeTreeData::PerShardDataParts reshardPartition( const ReshardingJob & job, DiskSpaceMonitor::Reservation * disk_reservation = nullptr); - /// Примерное количество места на диске, нужное для мерджа. С запасом. + /// The approximate amount of disk space needed for merge. With a surplus. static size_t estimateDiskSpaceForMerge(const MergeTreeData::DataPartsVector & parts); private: - /** Выбрать все куски принадлежащие одной партиции. + /** Select all parts belonging to the same partition. */ MergeTreeData::DataPartsVector selectAllPartsFromPartition(DayNum_t partition); @@ -145,7 +145,7 @@ private: Logger * log; - /// Когда в последний раз писали в лог, что место на диске кончилось (чтобы не писать об этом слишком часто). + /// When the last time you wrote to the log that the disk space was running out (not to write about this too often). time_t disk_space_warning_time = 0; CancellationHook cancellation_hook; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h index 5878ed4e181..e45b6442814 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataPart.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataPart.h @@ -13,7 +13,7 @@ namespace DB { -/// Чексумма одного файла. +/// Checksum of one file. struct MergeTreeDataPartChecksum { size_t file_size {}; @@ -34,8 +34,8 @@ struct MergeTreeDataPartChecksum }; -/** Контрольные суммы всех не временных файлов. - * Для сжатых файлов хранятся чексумма и размер разжатых данных, чтобы не зависеть от способа сжатия. +/** Checksums of all non-temporary files. + * For compressed files, the check sum and the size of the decompressed data are stored to not depend on the compression method. */ struct MergeTreeDataPartChecksums { @@ -48,15 +48,15 @@ struct MergeTreeDataPartChecksums void add(MergeTreeDataPartChecksums && rhs_checksums); - /// Проверяет, что множество столбцов и их контрольные суммы совпадают. Если нет - бросает исключение. - /// Если have_uncompressed, для сжатых файлов сравнивает чексуммы разжатых данных. Иначе сравнивает только чексуммы файлов. + /// Checks that the set of columns and their checksums are the same. If not, throws an exception. + /// If have_uncompressed, for compressed files it compares the checksums of the decompressed data. Otherwise, it compares only the checksums of the files. void checkEqual(const MergeTreeDataPartChecksums & rhs, bool have_uncompressed) const; - /// Проверяет, что в директории есть все нужные файлы правильных размеров. Не проверяет чексуммы. + /// Checks that the directory contains all the needed files of the correct size. Does not check the checksum. void checkSizes(const String & path) const; - /// Сериализует и десериализует в человекочитаемом виде. - bool read(ReadBuffer & in); /// Возвращает false, если чексуммы в слишком старом формате. + /// Serializes and deserializes in human readable form. + bool read(ReadBuffer & in); /// Returns false if the checksum is too old. bool read_v2(ReadBuffer & in); bool read_v3(ReadBuffer & in); bool read_v4(ReadBuffer & in); @@ -67,7 +67,7 @@ struct MergeTreeDataPartChecksums return files.empty(); } - /// Контрольная сумма от множества контрольных сумм .bin файлов. + /// Checksum from the set of checksums of .bin files. void summaryDataChecksum(SipHash & hash) const; String toString() const; @@ -78,7 +78,7 @@ struct MergeTreeDataPartChecksums class MergeTreeData; -/// Описание куска с данными. +/// Description of the data part. struct MergeTreeDataPart : public ActiveDataPartSet::Part { using Checksums = MergeTreeDataPartChecksums; @@ -106,12 +106,12 @@ struct MergeTreeDataPart : public ActiveDataPartSet::Part MergeTreeData & storage; size_t size = 0; /// in number of marks. - std::atomic size_in_bytes {0}; /// размер в байтах, 0 - если не посчитано; - /// используется из нескольких потоков без блокировок (изменяется при ALTER). + std::atomic size_in_bytes {0}; /// size in bytes, 0 - if not counted; + /// is used from several threads without locks (it is changed with ALTER). time_t modification_time = 0; - mutable time_t remove_time = std::numeric_limits::max(); /// Когда кусок убрали из рабочего набора. + mutable time_t remove_time = std::numeric_limits::max(); /// When the part is removed from the working set. - /// Если true, деструктор удалит директорию с куском. + /// If true, the destructor will delete the directory with the part. bool is_temp = false; /// For resharding. @@ -131,30 +131,30 @@ struct MergeTreeDataPart : public ActiveDataPartSet::Part using ColumnToSize = std::map; - /** Блокируется на запись при изменении columns, checksums или любых файлов куска. - * Блокируется на чтение при чтении columns, checksums или любых файлов куска. + /** It is blocked for writing when changing columns, checksums or any part files. + * Locked to read when reading columns, checksums or any part files. */ mutable Poco::RWLock columns_lock; - /** Берется на все время ALTER куска: от начала записи временных фалов до их переименования в постоянные. - * Берется при разлоченном columns_lock. + /** It is taken for the whole time ALTER a part: from the beginning of the recording of the temporary files to their renaming to permanent. + * It is taken with unlocked `columns_lock`. * - * NOTE: "Можно" было бы обойтись без этого мьютекса, если бы можно было превращать ReadRWLock в WriteRWLock, не снимая блокировку. - * Такое превращение невозможно, потому что создало бы дедлок, если делать его из двух потоков сразу. - * Взятие этого мьютекса означает, что мы хотим заблокировать columns_lock на чтение с намерением потом, не - * снимая блокировку, заблокировать его на запись. + * NOTE: "You can" do without this mutex if you could turn ReadRWLock into WriteRWLock without removing the lock. + * This transformation is impossible, because it would create a deadlock, if you do it from two threads at once. + * Taking this mutex means that we want to lock columns_lock on read with intention then, not + * unblocking, block it for writing. */ mutable std::mutex alter_mutex; ~MergeTreeDataPart(); - /// Вычисляем суммарный размер всей директории со всеми файлами + /// Calculate the total size of the entire directory with all the files static size_t calcTotalSize(const String & from); void remove() const; void renameTo(const String & new_name) const; - /// Переименовывает кусок, дописав к имени префикс. to_detached - также перенести в директорию detached. + /// Renames a piece by appending a prefix to the name. To_detached - also moved to the detached directory. void renameAddPrefix(bool to_detached, const String & prefix) const; /// Loads index file. Also calculates this->size if size=0 diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 5ccafdd2501..a19cde60f2d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -10,15 +10,15 @@ namespace DB class PKCondition; -/** Выполняет запросы SELECT на данных из merge-дерева. +/** Executes SELECT queries on data from the merge tree. */ class MergeTreeDataSelectExecutor { public: MergeTreeDataSelectExecutor(MergeTreeData & data_); - /** При чтении, выбирается набор кусков, покрывающий нужный диапазон индекса. - * max_block_number_to_read - если не ноль - не читать все куски, у которых правая граница больше этого порога. + /** When reading, selects a set of parts that covers the desired range of the index. + * max_block_number_to_read - if not zero, do not read all the parts whose right border is greater than this threshold. */ BlockInputStreams read( const Names & column_names, @@ -28,7 +28,7 @@ public: QueryProcessingStage::Enum & processed_stage, size_t max_block_size, unsigned threads, - size_t * inout_part_index, /// Если не nullptr, из этого счетчика берутся значения для виртуального столбца _part_index. + size_t * inout_part_index, /// If not nullptr, from this counter values are taken ​​for the virtual column _part_index. Int64 max_block_number_to_read) const; private: @@ -59,13 +59,13 @@ private: const Settings & settings, const Context & context) const; - /// Получить приблизительное значение (оценку снизу - только по полным засечкам) количества строк, попадающего под индекс. + /// Get the approximate value (bottom estimate - only by full marks) of the number of rows falling under the index. size_t getApproximateTotalRowsToRead( const MergeTreeData::DataPartsVector & parts, const PKCondition & key_condition, const Settings & settings) const; - /// Создать выражение "Sign == 1". + /// Create the expression "Sign == 1". void createPositiveSignCondition( ExpressionActionsPtr & out_expression, String & out_column, diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h index f0aedb0b046..66b38c9b1c2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -36,7 +36,7 @@ struct BlockWithDateInterval using BlocksWithDateIntervals = std::list; -/** Записывает новые куски с данными в merge-дерево. + /** Writes new parts of data to the merge tree. */ class MergeTreeDataWriter { diff --git a/dbms/src/Storages/MergeTree/MergeTreePartChecker.h b/dbms/src/Storages/MergeTree/MergeTreePartChecker.h index cf7adf1ec8c..9c503261af8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartChecker.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartChecker.h @@ -11,9 +11,9 @@ class MergeTreePartChecker public: struct Settings { - bool verbose = false; /// Пишет в stderr прогресс и ошибки, и не останавливается при первой ошибке. - bool require_checksums = false; /// Требует, чтобы был columns.txt. - bool require_column_files = false; /// Требует, чтобы для всех столбцов из columns.txt были файлы. + bool verbose = false; /// Writes progress and errors to stderr, and does not stop at the first error. + bool require_checksums = false; /// Requires column.txt to be. + bool require_column_files = false; /// Requires that all columns from columns.txt have files. size_t index_granularity = 8192; Settings & setVerbose(bool verbose_) { verbose = verbose_; return *this; } @@ -22,16 +22,16 @@ public: Settings & setIndexGranularity(size_t index_granularity_) { index_granularity = index_granularity_; return *this; } }; - /** Полностью проверяет данные кусочка: - * - Вычисляет контрольные суммы и сравнивает с checksums.txt. - * - Для массивов и строк проверяет соответствие размеров и количества данных. - * - Проверяет правильность засечек. - * Бросает исключение, если кусок испорчен или если проверить не получилось (TODO: можно попробовать разделить эти случаи). + /** Completely checks the part data + * - Calculates checksums and compares them with checksums.txt. + * - For arrays and strings, checks the correspondence of the size and amount of data. + * - Checks the correctness of marks. + * Throws an exception if the piece is corrupted or if the check fails (TODO: you can try to separate these cases). */ static void checkDataPart( String path, const Settings & settings, - const DataTypes & primary_key_data_types, /// Проверять первичный ключ. Если не надо - передайте пустой массив. + const DataTypes & primary_key_data_types, /// Check the primary key. If it is not necessary, pass an empty array. MergeTreeData::DataPart::Checksums * out_checksums = nullptr, std::atomic * is_cancelled = nullptr); }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeSharder.h b/dbms/src/Storages/MergeTree/MergeTreeSharder.h index 115a991da8d..bce2225f78a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeSharder.h +++ b/dbms/src/Storages/MergeTree/MergeTreeSharder.h @@ -26,7 +26,7 @@ using ShardedBlocksWithDateIntervals = std::list; struct ReshardingJob; -/** Создаёт новые шардированные куски с данными. +/** Creates new shard parts of data. */ class MergeTreeSharder final { @@ -35,10 +35,10 @@ public: MergeTreeSharder(const MergeTreeSharder &) = delete; MergeTreeSharder & operator=(const MergeTreeSharder &) = delete; - /** Разбивает блок на блоки по ключу шардирования, каждый из которых - * нужно записать в отдельный кусок. Работает детерминированно: если - * отдать на вход такой же блок, на выходе получатся такие же блоки в - * таком же порядке. + /** Breaks the block into blocks by the sharding key, each of which + * must be written to a separate part. It works deterministically: if + * give the same block to the input, the output will be the same blocks in the + * in the same order. */ ShardedBlocksWithDateIntervals shardBlock(const Block & block); diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index f63cffc190c..01da66151f6 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -65,7 +65,7 @@ protected: void addStream(const String & path, const String & name, const IDataType & type, size_t estimated_size, size_t level, const String & filename, bool skip_offsets); - /// Записать данные одного столбца. + /// Write data of one column. void writeData(const String & name, const IDataType & type, const IColumn & column, OffsetColumns & offset_columns, size_t level, bool skip_offsets); @@ -73,7 +73,7 @@ protected: ColumnStreams column_streams; - /// Смещение до первой строчки блока, для которой надо записать индекс. + /// The offset to the first row of the block for which you want to write the index. size_t index_offset = 0; size_t min_compress_block_size; @@ -90,8 +90,8 @@ private: }; -/** Для записи одного куска. - * Данные относятся к одному месяцу, и пишутся в один кускок. +/** To write one part. + * The data refers to one month, and are written in one part. */ class MergedBlockOutputStream : public IMergedBlockOutputStream { @@ -112,11 +112,11 @@ public: std::string getPartPath() const; - /// Если данные заранее отсортированы. + /// If the data is pre-sorted. void write(const Block & block) override; - /** Если данные не отсортированы, но мы заранее вычислили перестановку, после которой они станут сортированными. - * Этот метод используется для экономии оперативки, так как не нужно держать одновременно два блока - исходный и отсортированный. + /** If the data is not sorted, but we have previously calculated the permutation, after which they will be sorted. + * This method is used to save RAM, since you do not need to keep two blocks at once - the original one and the sorted one. */ void writeWithPermutation(const Block & block, const IColumn::Permutation * permutation); @@ -130,14 +130,14 @@ public: MergeTreeData::DataPart::Index & getIndex(); - /// Сколько засечек уже записано. + /// How many marks are already written. size_t marksCount(); private: void init(); - /** Если задана permutation, то переставляет значения в столбцах при записи. - * Это нужно, чтобы не держать целый блок в оперативке для его сортировки. + /** If `permutation` is given, it rearranges the values ​​in the columns when writing. + * This is necessary to not keep the whole block in the RAM to sort it. */ void writeImpl(const Block & block, const IColumn::Permutation * permutation); @@ -153,7 +153,7 @@ private: }; -/// Записывает только те, столбцы, что лежат в block +/// Writes only those columns that are in `block` class MergedColumnOnlyOutputStream : public IMergedBlockOutputStream { public: diff --git a/dbms/src/Storages/MergeTree/PKCondition.h b/dbms/src/Storages/MergeTree/PKCondition.h index 93362ebd165..70f9c3dff31 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.h +++ b/dbms/src/Storages/MergeTree/PKCondition.h @@ -17,7 +17,7 @@ class IFunction; using FunctionPtr = std::shared_ptr; -/** Диапазон с открытыми или закрытыми концами; возможно, неограниченный. +/** Range with open or closed ends; Perhaps unlimited. */ struct Range { @@ -26,21 +26,21 @@ private: static bool less(const Field & lhs, const Field & rhs); public: - Field left; /// левая граница, если есть - Field right; /// правая граница, если есть - bool left_bounded = false; /// ограничен ли слева - bool right_bounded = false; /// ограничен ли справа - bool left_included = false; /// включает левую границу, если есть - bool right_included = false; /// включает правую границу, если есть + Field left; /// the left border, if any + Field right; /// the right border, if any + bool left_bounded = false; /// limited to the left + bool right_bounded = false; /// limited to the right + bool left_included = false; /// includes the left border, if any + bool right_included = false; /// includes the right border, if any - /// Всё множество. + /// The whole set. Range() {} - /// Одна точка. + /// One point. Range(const Field & point) : left(point), right(point), left_bounded(true), right_bounded(true), left_included(true), right_included(true) {} - /// Ограниченный с двух сторон диапазон. + /// A bounded two-sided range. Range(const Field & left_, bool left_included_, const Field & right_, bool right_included_) : left(left_), right(right_), left_bounded(true), right_bounded(true), @@ -69,9 +69,9 @@ public: return r; } - /** Оптимизировать диапазон. Если у него есть открытая граница и тип Field "неплотный" - * - то преобразовать её в закрытую, сузив на единицу. - * То есть, например, превратить (0,2) в [1]. + /** Optimize the range. If it has an open boundary and the Field type is "loose" + * - then convert it to closed, narrowing by one. + * That is, for example, turn (0,2) into [1]. */ void shrinkToIncludedIfPossible() { @@ -110,13 +110,13 @@ public: || ((!left_included || !right_included) && !less(left, right))); } - /// x входит в range + /// x contained in the range bool contains(const Field & x) const { return !leftThan(x) && !rightThan(x); } - /// x находится левее + /// x is to the left bool rightThan(const Field & x) const { return (left_bounded @@ -124,7 +124,7 @@ public: : false); } - /// x находится правее + /// x is to the right bool leftThan(const Field & x) const { return (right_bounded @@ -134,7 +134,7 @@ public: bool intersectsRange(const Range & r) const { - /// r левее меня. + /// r to the left of me. if (r.right_bounded && left_bounded && (less(r.right, left) @@ -142,11 +142,11 @@ public: && equals(r.right, left)))) return false; - /// r правее меня. + /// r to the right of me. if (r.left_bounded && right_bounded && (less(right, r.left) /// ...} {... - || ((!right_included || !r.left_included) /// ...)[... или ...](... + || ((!right_included || !r.left_included) /// ...) [... or ...] (... && equals(r.left, right)))) return false; @@ -155,7 +155,7 @@ public: bool containsRange(const Range & r) const { - /// r начинается левее меня. + /// r starts to the left of me. if (left_bounded && (!r.left_bounded || less(r.left, left) @@ -164,7 +164,7 @@ public: && equals(r.left, left)))) return false; - /// r заканчивается правее меня. + /// r ends right of me. if (right_bounded && (!r.right_bounded || less(right, r.right) @@ -190,59 +190,59 @@ public: class ASTSet; -/** Условие на индекс. +/** Condition on the index. * - * Состоит из условий на принадлежность ключа всевозможным диапазонам или множествам, - * а также логических связок AND/OR/NOT над этими условиями. + * Consists of the conditions for the key belonging to all possible ranges or sets, + * as well as logical links AND/OR/NOT above these conditions. * - * Составляет reverse polish notation от этих условий - * и умеет вычислять (интерпретировать) её выполнимость над диапазонами ключа. + * Constructs a reverse polish notation from these conditions + * and can calculate (interpret) its feasibility over key ranges. */ class PKCondition { public: - /// Не учитывает секцию SAMPLE. all_columns - набор всех столбцов таблицы. + /// Does not include the SAMPLE section. all_columns - the set of all columns of the table. PKCondition(ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr, const Block & pk_sample_block); - /// Выполнимо ли условие в диапазоне ключей. - /// left_pk и right_pk должны содержать все поля из sort_descr в соответствующем порядке. - /// data_types - типы столбцов первичного ключа. + /// Whether the condition is feasible in the key range. + /// left_pk and right_pk must contain all fields in the sort_descr in the appropriate order. + /// data_types - the types of the primary key columns. bool mayBeTrueInRange(size_t used_key_size, const Field * left_pk, const Field * right_pk, const DataTypes & data_types) const; - /// Выполнимо ли условие в полубесконечном (не ограниченном справа) диапазоне ключей. - /// left_pk должен содержать все поля из sort_descr в соответствующем порядке. + /// Is the condition valid in a semi-infinite (not limited to the right) key range. + /// left_pk must contain all the fields in the sort_descr in the appropriate order. bool mayBeTrueAfter(size_t used_key_size, const Field * left_pk, const DataTypes & data_types) const; - /// Проверяет, что индекс не может быть использован. + /// Checks that the index can not be used. bool alwaysUnknownOrTrue() const; - /// Получить максимальный номер используемого в условии элемента первичного ключа. + /// Get the maximum number of the primary key element used in the condition. size_t getMaxKeyColumn() const; - /// Наложить дополнительное условие: значение в столбце column должно быть в диапазоне range. - /// Возвращает, есть ли такой столбец в первичном ключе. + /// Impose an additional condition: the value in the column column must be in the `range` range. + /// Returns whether there is such a column in the primary key. bool addCondition(const String & column, const Range & range); String toString() const; - /// Выражение хранится в виде обратной польской строки (Reverse Polish Notation). + /// The expression is stored as Reverse Polish Notation. struct RPNElement { enum Function { - /// Атомы логического выражения. + /// Atoms of a Boolean expression. FUNCTION_IN_RANGE, FUNCTION_NOT_IN_RANGE, FUNCTION_IN_SET, FUNCTION_NOT_IN_SET, - FUNCTION_UNKNOWN, /// Может принимать любое значение. - /// Операторы логического выражения. + FUNCTION_UNKNOWN, /// Can take any value. + /// Operators of the logical expression. FUNCTION_NOT, FUNCTION_AND, FUNCTION_OR, - /// Константы + /// Constants ALWAYS_FALSE, ALWAYS_TRUE, }; @@ -257,18 +257,18 @@ public: Function function = FUNCTION_UNKNOWN; - /// Для FUNCTION_IN_RANGE и FUNCTION_NOT_IN_RANGE. + /// For FUNCTION_IN_RANGE and FUNCTION_NOT_IN_RANGE. Range range; size_t key_column; - /// Для FUNCTION_IN_SET, FUNCTION_NOT_IN_SET + /// For FUNCTION_IN_SET, FUNCTION_NOT_IN_SET ASTPtr in_function; - /** Цепочка возможно-монотонных функций. - * Если столбец первичного ключа завёрнут в функции, которые могут быть монотонными в некоторых диапазонах значений - * (например: -toFloat64(toDayOfWeek(date))), то здесь будут расположены функции: toDayOfWeek, toFloat64, negate. + /** A chain of possibly monotone functions. + * If the primary key column is wrapped in functions that can be monotonous in some value ranges + * (for example: -toFloat64(toDayOfWeek(date))), then here the functions will be located: toDayOfWeek, toFloat64, negate. */ using MonotonicFunctionsChain = std::vector; - mutable MonotonicFunctionsChain monotonic_functions_chain; /// Выполнение функции не нарушает константность. + mutable MonotonicFunctionsChain monotonic_functions_chain; /// The function execution does not violate the constancy. }; static Block getBlockWithConstants( diff --git a/dbms/src/Storages/MergeTree/RemoteDiskSpaceMonitor.h b/dbms/src/Storages/MergeTree/RemoteDiskSpaceMonitor.h index 64249e3328c..12a0e296530 100644 --- a/dbms/src/Storages/MergeTree/RemoteDiskSpaceMonitor.h +++ b/dbms/src/Storages/MergeTree/RemoteDiskSpaceMonitor.h @@ -12,7 +12,7 @@ class Context; namespace RemoteDiskSpaceMonitor { -/** Сервис для получения информации о свободном месте на диске. +/** Service to get information about free disk space. */ class Service final : public InterserverIOEndpoint { @@ -27,7 +27,7 @@ private: const Context & context; }; -/** Клиент для получения информации о свободном месте на удалённом диске. +/** Client to get information about free space on a remote disk. */ class Client final { diff --git a/dbms/src/Storages/MergeTree/RemoteQueryExecutor.h b/dbms/src/Storages/MergeTree/RemoteQueryExecutor.h index 7e6c1fefb88..93a93b74383 100644 --- a/dbms/src/Storages/MergeTree/RemoteQueryExecutor.h +++ b/dbms/src/Storages/MergeTree/RemoteQueryExecutor.h @@ -11,7 +11,7 @@ class Context; namespace RemoteQueryExecutor { -/** Сервис для выполнения SQL запросов. +/** Service for executing SQL queries. */ class Service final : public InterserverIOEndpoint { @@ -26,7 +26,7 @@ private: Context & context; }; -/** Клиент для удалённого выполнения SQL запросов. +/** Client for remote execution of SQL queries. */ class Client final { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAddress.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAddress.h index 32c1219f822..e1447b33f39 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAddress.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAddress.h @@ -8,7 +8,7 @@ namespace DB { -/// Позволяет узнать, куда отправлять запросы, чтобы попасть на реплику. +/// Lets you know where to send requests to get to the replica. struct ReplicatedMergeTreeAddress { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h index 10ef0121e12..a5830a8ebcf 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.h @@ -12,10 +12,10 @@ namespace DB class StorageReplicatedMergeTree; -/** Следит за изменением структуры таблицы в ZooKeeper и выполняет необходимые преобразования. +/** Keeps track of changing the table structure in ZooKeeper and performs the necessary conversions. * - * NOTE Это не имеет отношения к манипуляциям с партициями, - * которые обрабатываются через очередь репликации. + * NOTE This has nothing to do with manipulating partitions, + * which are processed through the replication queue. */ class ReplicatedMergeTreeAlterThread { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index aff79c1eade..86caa13fc2d 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -10,7 +10,7 @@ namespace DB class StorageReplicatedMergeTree; -/** Удаляет устаревшие данные таблицы типа ReplicatedMergeTree. +/** Removes obsolete data from a table of type ReplicatedMergeTree. */ class ReplicatedMergeTreeCleanupThread { @@ -31,17 +31,17 @@ private: void run(); void iterate(); - /// Удалить старые куски с диска и из ZooKeeper. + /// Delete old chunks from disk and from ZooKeeper. void clearOldParts(); - /// Удалить из ZooKeeper старые записи в логе. + /// Remove old records from ZooKeeper. void clearOldLogs(); - /// Удалить из ZooKeeper старые хеши блоков. Это делает ведущая реплика. + /// Remove old block hashes from ZooKeeper. This makes a leading replica. void clearOldBlocks(); - /// TODO Удаление старых quorum/failed_parts - /// TODO Удаление старых nonincrement_block_numbers + /// TODO Removing old quorum/failed_parts + /// TODO Removing old nonincrement_block_numbers }; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 1a185ef9eb6..fa2c2436b23 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -26,16 +26,16 @@ namespace ErrorCodes } -/// Запись о том, что нужно сделать. Только данные (их можно копировать). +/// Record about what needs to be done. Only data (you can copy them). struct ReplicatedMergeTreeLogEntryData { enum Type { - EMPTY, /// Не используется. - GET_PART, /// Получить кусок с другой реплики. - MERGE_PARTS, /// Слить куски. - DROP_RANGE, /// Удалить куски в указанном месяце в указанном диапазоне номеров. - ATTACH_PART, /// Перенести кусок из директории detached или unreplicated. + EMPTY, /// Not used. + GET_PART, /// Get the part from another replica. + MERGE_PARTS, /// Merge the parts. + DROP_RANGE, /// Delete the parts in the specified month in the specified number range. + ATTACH_PART, /// Move a part from the `detached` or `unreplicated` directory. }; String typeToString() const @@ -58,38 +58,38 @@ struct ReplicatedMergeTreeLogEntryData String znode_name; Type type = EMPTY; - String source_replica; /// Пустая строка значит, что эта запись была добавлена сразу в очередь, а не скопирована из лога. + String source_replica; /// Empty string means that this entry was added to the queue immediately, and not copied from the log. - /// Имя куска, получающегося в результате. - /// Для DROP_RANGE имя несуществующего куска. Нужно удалить все куски, покрытые им. + /// The name of resulting part. + /// For DROP_RANGE, the name of a non-existent part. You need to remove all the parts covered by it. String new_part_name; - String block_id; /// Для кусков нулевого уровня - идентификатор блока для дедупликации (имя ноды в /blocks/). + String block_id; /// For parts of level zero, the block identifier for deduplication (node ​​name in /blocks /). Strings parts_to_merge; - /// Для DROP_RANGE, true значит, что куски нужно не удалить, а перенести в директорию detached. + /// For DROP_RANGE, true means that the parts need not be deleted, but moved to the `detached` directory. bool detach = false; - /// Для ATTACH_PART имя куска в директории detached или unreplicated. + /// For ATTACH_PART, the name of the part in the `detached` or `unreplicated` directory. String source_part_name; - /// Нужно переносить из директории unreplicated, а не detached. + /// Must be moved from the `unreplicated` directory, not `detached`. bool attach_unreplicated = false; - /// Доступ под queue_mutex, см. ReplicatedMergeTreeQueue. - bool currently_executing = false; /// Выполняется ли действие сейчас. - /// Эти несколько полей имеют лишь информационный характер (для просмотра пользователем с помощью системных таблиц). - /// Доступ под queue_mutex, см. ReplicatedMergeTreeQueue. - size_t num_tries = 0; /// Количество попыток выполнить действие (с момента старта сервера; включая выполняющееся). - std::exception_ptr exception; /// Последний эксепшен, в случае безуспешной попытки выполнить действие. - time_t last_attempt_time = 0; /// Время начала последней попытки выполнить действие. - size_t num_postponed = 0; /// Количество раз, когда действие было отложено. - String postpone_reason; /// Причина, по которой действие было отложено, если оно отложено. - time_t last_postpone_time = 0; /// Время последнего раза, когда действие было отложено. + /// Access under queue_mutex, see ReplicatedMergeTreeQueue. + bool currently_executing = false; /// Whether the action is executing now. + /// These several fields are informational only (for viewing by the user using system tables). + /// Access under queue_mutex, see ReplicatedMergeTreeQueue. + size_t num_tries = 0; /// The number of attempts to perform the action (since the server started, including the running one). + std::exception_ptr exception; /// The last exception, in the case of an unsuccessful attempt to perform the action. + time_t last_attempt_time = 0; /// The time at which the last attempt was attempted to complete the action. + size_t num_postponed = 0; /// The number of times the action was postponed. + String postpone_reason; /// The reason why the action was postponed, if it was postponed. + time_t last_postpone_time = 0; /// The time of the last time the action was postponed. - /// Время создания или время копирования из общего лога в очередь конкретной реплики. + /// Creation time or the time to copy from the general log to the queue of a particular replica. time_t create_time = 0; - /// Величина кворума (для GET_PART) - ненулевое значение при включенной кворумной записи. + /// The quorum value (for GET_PART) is a non-zero value when the quorum write is enabled. size_t quorum = 0; }; @@ -98,7 +98,7 @@ struct ReplicatedMergeTreeLogEntry : ReplicatedMergeTreeLogEntryData { using Ptr = std::shared_ptr; - std::condition_variable execution_complete; /// Пробуждается когда currently_executing становится false. + std::condition_variable execution_complete; /// Awake when currently_executing becomes false. static Ptr parse(const String & s, const Stat & stat); }; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h index 7a3a2259038..bc91303cdd6 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.h @@ -18,19 +18,19 @@ namespace DB class StorageReplicatedMergeTree; -/** Проверяет целостность кусков, запрошенных для проверки. +/** Checks the integrity of the parts requested for validation. * - * Определяет лишние куски и убирает их из рабочего набора. - * Находит отсутствующие куски и добавляет их для скачивания с реплик. - * Проверяет целостность данных и, в случае нарушения, - * убирает кусок из рабочего набора и добавляет для скачивания с реплик. + * Identifies the extra parts and removes them from the working set. + * Find the missing parts and add them for download from replicas. + * Checks the integrity of the data and, in the event of a violation, + * removes a part from the working set and adds it for download from replicas. */ class ReplicatedMergeTreePartCheckThread { public: ReplicatedMergeTreePartCheckThread(StorageReplicatedMergeTree & storage_); - /// Разбор очереди для проверки осуществляется в фоновом потоке, который нужно сначала запустить. + /// Processing of the queue to be checked is done in the background thread, which you must first start. void start(); void stop(); @@ -58,11 +58,11 @@ public: TemporarilyStop temporarilyStop() { return TemporarilyStop(this); } - /// Добавить кусок (для которого есть подозрения, что он отсутствует, повреждён или не нужен) в очередь для проверки. - /// delay_to_check_seconds - проверять не раньше чем через указанное количество секунд. + /// Add a part (for which there are suspicions that it is missing, damaged or not needed) in the queue for check. + /// delay_to_check_seconds - check no sooner than the specified number of seconds. void enqueuePart(const String & name, time_t delay_to_check_seconds = 0); - /// Получить количество кусков в очереди для проверки. + /// Get the number of parts in the queue for check. size_t size() const; ~ReplicatedMergeTreePartCheckThread() @@ -80,12 +80,12 @@ private: Logger * log; using StringSet = std::set; - using PartToCheck = std::pair; /// Имя куска и минимальное время для проверки (или ноль, если не важно). + using PartToCheck = std::pair; /// The name of the part and the minimum time to check (or zero, if not important). using PartsToCheckQueue = std::list; - /** Куски, для которых нужно проверить одно из двух: - * - Если кусок у нас есть, сверить, его данные с его контрольными суммами, а их с ZooKeeper. - * - Если куска у нас нет, проверить, есть ли он (или покрывающий его кусок) хоть у кого-то. + /** Parts for which you want to check one of two: + * - If we have the part, check, its data with its checksums, and them with ZooKeeper. + * - If we do not have a part, check to see if it (or the part covering it) exists anywhere. */ StringSet parts_set; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 128b743e83f..80eee2e6338 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -33,7 +33,7 @@ private: } }; - /// Для вычисления min_unprocessed_insert_time, max_processed_insert_time, по которым вычисляется отставание реплик. + /// To calculate min_unprocessed_insert_time, max_processed_insert_time, for which the replica lag is calculated. using InsertsByTime = std::set; @@ -41,8 +41,8 @@ private: String replica_path; String logger_name; - /** Очередь того, что нужно сделать на этой реплике, чтобы всех догнать. Берется из ZooKeeper (/replicas/me/queue/). - * В ZK записи в хронологическом порядке. Здесь - не обязательно. + /** The queue of what you need to do on this line to catch up. It is taken from ZooKeeper (/replicas/me/queue/). + * In ZK records in chronological order. Here it is not necessary. */ Queue queue; @@ -52,51 +52,51 @@ private: time_t last_queue_update = 0; - /// Куски, которые появятся в результате действий, выполняемых прямо сейчас фоновыми потоками (этих действий нет в очереди). - /// Используется, чтобы не выполнять в тот же момент другие действия с этими кусками. + /// parts that will appear as a result of actions performed right now by background threads (these actions are not in the queue). + /// Used to not perform other actions at the same time with these parts. StringSet future_parts; - /// На доступ к queue, future_parts, ... + /// To access the queue, future_parts, ... std::mutex mutex; - /// Обеспечивает только один одновременный вызов pullLogsToQueue. + /// Provides only one simultaneous call to pullLogsToQueue. std::mutex pull_logs_to_queue_mutex; - /** Каким будет множество активных кусков после выполнения всей текущей очереди - добавления новых кусков и выполнения слияний. - * Используется, чтобы определять, какие мерджи уже были назначены: - * - если в этом множестве есть кусок, то мерджи более мелких кусков внутри его диапазона не делаются. - * Дополнительно, сюда также добавляются специальные элементы, чтобы явно запретить мерджи в некотором диапазоне (см. disableMergesInRange). - * Это множество защищено своим mutex-ом. + /** What will be the set of active parts after running the entire current queue - adding new parts and performing merges. + * Used to determine which merges have already been assigned: + * - if there is a part in this set, then the smaller parts inside its range are not made. + * Additionally, special elements are also added here to explicitly disallow the merge in a certain range (see disableMergesInRange). + * This set is protected by its mutex. */ ActiveDataPartSet virtual_parts; Logger * log = nullptr; - /// Положить набор (уже существующих) кусков в virtual_parts. + /// Put a set of (already existing) parts in virtual_parts. void initVirtualParts(const MergeTreeData::DataParts & parts); - /// Загрузить (инициализировать) очередь из ZooKeeper (/replicas/me/queue/). + /// Load (initialize) a queue from ZooKeeper (/replicas/me/queue/). void load(zkutil::ZooKeeperPtr zookeeper); void insertUnlocked(LogEntryPtr & entry); void remove(zkutil::ZooKeeperPtr zookeeper, LogEntryPtr & entry); - /** Можно ли сейчас попробовать выполнить это действие. Если нет, нужно оставить его в очереди и попробовать выполнить другое. - * Вызывается под queue_mutex. + /** Can I now try this action. If not, you need to leave it in the queue and try another one. + * Called under queue_mutex. */ bool shouldExecuteLogEntry(const LogEntry & entry, String & out_postpone_reason, MergeTreeDataMerger & merger, MergeTreeData & data); - /// После удаления элемента очереди, обновить времена insert-ов в оперативке. Выполняется под queue_mutex. - /// Возвращает информацию, какие времена изменились - эту информацию можно передать в updateTimesInZooKeeper. + /// After removing the queue element, update the insertion times in the RAM. Running under queue_mutex. + /// Returns information about what times have changed - this information can be passed to updateTimesInZooKeeper. void updateTimesOnRemoval(const LogEntryPtr & entry, bool & min_unprocessed_insert_time_changed, bool & max_processed_insert_time_changed); - /// Обновить времена insert-ов в ZooKeeper. + /// Update the insertion times in ZooKeeper. void updateTimesInZooKeeper(zkutil::ZooKeeperPtr zookeeper, bool min_unprocessed_insert_time_changed, bool max_processed_insert_time_changed); - /// Помечает элемент очереди как выполняющийся. + /// Marks the element of the queue as running. class CurrentlyExecuting { private: @@ -105,7 +105,7 @@ private: friend class ReplicatedMergeTreeQueue; - /// Создаётся только в функции selectEntryToProcess. Вызывается под mutex-ом. + /// Created only in the selectEntryToProcess function. It is called under mutex. CurrentlyExecuting(ReplicatedMergeTreeQueue::LogEntryPtr & entry, ReplicatedMergeTreeQueue & queue); public: ~CurrentlyExecuting(); @@ -117,55 +117,55 @@ public: void initialize(const String & zookeeper_path_, const String & replica_path_, const String & logger_name_, const MergeTreeData::DataParts & parts, zkutil::ZooKeeperPtr zookeeper); - /** Вставить действие в конец очереди. - * Для восстановления битых кусков во время работы. - * Не вставляет само действие в ZK (сделайте это самостоятельно). + /** Paste action to the end of the queue. + * To restore broken parts during operation. + * Do not insert the action itself into ZK (do it yourself). */ void insert(zkutil::ZooKeeperPtr zookeeper, LogEntryPtr & entry); - /** Удалить действие с указанным куском (в качестве new_part_name) из очереди. - * Вызывается для невыполнимых действий в очереди - старых потерянных кусков. + /** Delete the action with the specified part (as new_part_name) from the queue. + * Called for unreachable actions in the queue - old lost parts. */ bool remove(zkutil::ZooKeeperPtr zookeeper, const String & part_name); - /** Скопировать новые записи из общего лога в очередь этой реплики. Установить log_pointer в соответствующее значение. - * Если next_update_event != nullptr, вызовет это событие, когда в логе появятся новые записи. - * Возвращает true, если новые записи были. + /** Copy the new entries from the shared log to the queue of this replica. Set the log_pointer to the appropriate value. + * If next_update_event != nullptr, will call this event when new entries appear in the log. + * Returns true if new entries have been. */ bool pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, zkutil::EventPtr next_update_event); - /** Удалить из очереди действия с кусками, покрываемыми part_name (из ZK и из оперативки). - * А также дождаться завершения их выполнения, если они сейчас выполняются. + /** Remove the action from the queue with the parts covered by part_name (from ZK and from the RAM). + * And also wait for the completion of their execution, if they are now being executed. */ void removeGetsAndMergesInRange(zkutil::ZooKeeperPtr zookeeper, const String & part_name); - /** В случае, когда для выполнения мерджа в part_name недостаёт кусков - * - переместить действия со сливаемыми кусками в конец очереди - * (чтобы раньше скачать готовый смердженный кусок с другой реплики). + /** In the case where there are not enough parts to perform the merge in part_name + * - move actions with merged parts to the end of the queue + * (in order to download a already merged part from another replica). */ StringSet moveSiblingPartsForMergeToEndOfQueue(const String & part_name); - /** Выбрать следующее действие для обработки. - * merger используется только чтобы проверить, не приостановлены ли мерджи. + /** Select the next action to process. + * merger is used only to check if the merges is not suspended. */ using SelectedEntry = std::pair>; SelectedEntry selectEntryToProcess(MergeTreeDataMerger & merger, MergeTreeData & data); - /** Выполнить функцию func для обработки действия. - * При этом, на время выполнения, отметить элемент очереди как выполняющийся - * (добавить в future_parts и другое). - * Если в процессе обработки было исключение - сохраняет его в entry. - * Возвращает true, если в процессе обработки не было исключений. + /** Execute `func` function to handle the action. + * In this case, at runtime, mark the queue element as running + * (add into future_parts and more). + * If there was an exception during processing, it saves it in `entry`. + * Returns true if there were no exceptions during the processing. */ bool processEntry(std::function get_zookeeper, LogEntryPtr & entry, const std::function func); - /// Будет ли кусок в будущем слит в более крупный (или мерджи кусков в данном диапазоне запрещены)? + /// Will a part in the future be merged into a larger part (or merges of parts in this range are prohibited)? bool partWillBeMergedOrMergesDisabled(const String & part_name) const; - /// Запретить слияния в указанном диапазоне. + /// Prohibit merges in the specified range. void disableMergesInRange(const String & part_name); - /// Посчитать количество слияний в очереди. + /// Count the number of merges in the queue. size_t countMerges(); struct Status @@ -182,21 +182,21 @@ public: UInt32 last_queue_update; }; - /// Получить информацию об очереди. + /// Get information about the queue. Status getStatus(); - /// Получить данные элементов очереди. + /// Get the data of the queue elements. using LogEntriesData = std::vector; void getEntries(LogEntriesData & res); - /// Получить информацию о временах insert-ов. + /// Get information about the insertion times. void getInsertTimes(time_t & out_min_unprocessed_insert_time, time_t & out_max_processed_insert_time) const; }; -/** Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper. - * Поддерживаются также отрицательные числа - для них имя ноды выглядит несколько глупо - * и не соответствует никакой автоинкрементной ноде в ZK. +/** Convert a number to a string in the format of the suffixes of auto-incremental nodes in ZooKeeper. + * Negative numbers are also supported - for them the name of the node looks somewhat silly + * and does not match any auto-incremented node in ZK. */ String padIndex(Int64 index); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h index 0270a73d420..d2ab304f02d 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h @@ -13,9 +13,9 @@ namespace DB { -/** Для реализации функциональности "кворумная запись". - * Информация о том, на каких репликах появился вставленный кусок данных, - * и на скольких репликах он должен быть. +/** To implement the functionality of the "quorum write". + * Information about which replicas the inserted part of data appeared on, + * and on how many replicas it should be. */ struct ReplicatedMergeTreeQuorumEntry { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h index 3226a4e9cf6..42f192ae909 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h @@ -13,10 +13,10 @@ namespace DB class StorageReplicatedMergeTree; -/** Инициализирует сессию в ZK. - * Выставляет эфемерные ноды. Выставляет нужные для обнаружения реплики значения нод. - * Запускает участие в выборе лидера. Запускает все фоновые потоки. - * Затем следит за тем, не истекла ли сессия. И если истекла - переинициализирует её. +/** Initializes ZK session. + * Exposes ephemeral nodes. It sets the node values ​​that are required for replica detection. + * Starts participation in the leader selection. Starts all background threads. + * Then monitors whether the session has expired. And if it expired, it will reinitialize it. */ class ReplicatedMergeTreeRestartingThread { @@ -51,23 +51,23 @@ private: Poco::Event wakeup_event; std::atomic need_stop {false}; - /// Случайные данные, которые мы записали в /replicas/me/is_active. + /// The random data we wrote into `/replicas/me/is_active`. String active_node_identifier; std::thread thread; void run(); - /// Запустить или остановить фоновые потоки. Используется для частичной переинициализации при пересоздании сессии в ZooKeeper. - bool tryStartup(); /// Возвращает false, если недоступен ZooKeeper. + /// Start or stop background threads. Used for partial reinitialization when re-creating a session in ZooKeeper. + bool tryStartup(); /// Returns false if ZooKeeper is not available. - /// Отметить в ZooKeeper, что эта реплика сейчас активна. + /// Note in ZooKeeper that this replica is currently active. void activateReplica(); - /// Удалить куски, для которых кворум пофейлился (за то время, когда реплика была неактивной). + /// Delete the parts for which the quorum has failed (for the time when the replica was inactive). void removeFailedQuorumParts(); - /// Если есть недостигнутый кворум, и у нас есть кусок, то добавить эту реплику в кворум. + /// If there is an unreachable quorum, and we have a part, then add this replica to the quorum. void updateQuorumIfWeHavePart(); void partialShutdown(); diff --git a/dbms/src/Storages/MergeTree/ReshardingJob.h b/dbms/src/Storages/MergeTree/ReshardingJob.h index e5caee37b3e..cd1c3f8fbf0 100644 --- a/dbms/src/Storages/MergeTree/ReshardingJob.h +++ b/dbms/src/Storages/MergeTree/ReshardingJob.h @@ -10,14 +10,14 @@ namespace DB class StorageReplicatedMergeTree; -/** Описание задачи перешардирования. +/** Description of the task of rescheduling. */ struct ReshardingJob final { public: ReshardingJob() = default; - /// Создаёт описание на основе его сериализованного представления. + /// Creates a description based on its serialized representation. ReshardingJob(const std::string & serialized_job); ReshardingJob(const std::string & database_name_, const std::string & table_name_, @@ -32,7 +32,7 @@ public: operator bool() const; - /// Сериализует описание задачи. + /// Serializes the task description. std::string toString() const; bool isCoordinated() const; diff --git a/dbms/src/Storages/MergeTree/ReshardingWorker.h b/dbms/src/Storages/MergeTree/ReshardingWorker.h index d62076251eb..46a37dc6c82 100644 --- a/dbms/src/Storages/MergeTree/ReshardingWorker.h +++ b/dbms/src/Storages/MergeTree/ReshardingWorker.h @@ -73,7 +73,7 @@ public: UInt64 subscribe(const std::string & coordinator_id, const std::string & query); /// Cancel the aforementionned subscription. void unsubscribe(const std::string & coordinator_id); - /// Увеличить количество партиций входящих в одну распределённую задачу. Вызывается с исполнителя. + /// Increase the number of partitions included in one distributed task. Called from the executor. void addPartitions(const std::string & coordinator_id, const PartitionList & partition_list); /// Rearrange partitions into two categories: coordinated job, uncoordinated job. /// Returns an iterator to the beginning of the list of uncoordinated jobs. @@ -196,9 +196,9 @@ private: /// Perform one job. void perform(const std::string & job_descriptor, const std::string & job_name); - /// Разбить куски входящие в партицию на несколько, согласно ключу шардирования. - /// Оновременно перегруппировать эти куски по шардам и слить куски в каждой группе. - /// При завершении этого процесса создаётся новая партиция для каждого шарда. + /// Split the parts of the partition into several, according to the sharding key. + /// Simultaneously regroup these parts by shards and merge the parts in each group. + /// When this process is completed, a new partition is created for each shard. void createShardedPartitions(); /// Upload all the partitions resulting from source partition resharding to their diff --git a/dbms/src/Storages/MergeTree/ShardedPartitionUploader.h b/dbms/src/Storages/MergeTree/ShardedPartitionUploader.h index be6a185fb94..44ccfd98964 100644 --- a/dbms/src/Storages/MergeTree/ShardedPartitionUploader.h +++ b/dbms/src/Storages/MergeTree/ShardedPartitionUploader.h @@ -14,7 +14,7 @@ class StorageReplicatedMergeTree; namespace ShardedPartitionUploader { -/** Сервис для получения кусков из партиции таблицы *MergeTree. +/** Service for retrieving parts from the partitions of the *MergeTree table. */ class Service final : public InterserverIOEndpoint { @@ -31,7 +31,7 @@ private: Logger * log = &Logger::get("ShardedPartitionUploader::Service"); }; -/** Клиент для отправления кусков из партиции таблицы *MergeTree. +/** Client for sending parts from the partition of the *MergeTree table. */ class Client final { diff --git a/dbms/src/Storages/StorageBuffer.h b/dbms/src/Storages/StorageBuffer.h index 788406c71d7..0ad84c59a52 100644 --- a/dbms/src/Storages/StorageBuffer.h +++ b/dbms/src/Storages/StorageBuffer.h @@ -18,23 +18,23 @@ namespace DB class Context; -/** При вставке, буферизует данные в оперативке, пока не превышены некоторые пороги. - * Когда пороги превышены - сбрасывает данные в другую таблицу. - * При чтении, читает как из своих буферов, так и из подчинённой таблицы. +/** During insertion, buffers the data in the RAM until certain thresholds are exceeded. + * When thresholds are exceeded, flushes the data to another table. + * When reading, it reads both from its buffers and from the subordinate table. * - * Буфер представляет собой набор из num_shards блоков. - * При записи, выбирается номер блока по остатку от деления ThreadNumber на num_shards (или один из других), - * и в соответствующий блок добавляются строчки. - * При использовании блока, он блокируется некоторым mutex-ом. Если при записи, соответствующий блок уже занят - * - пробуем заблокировать следующий по кругу блок, и так не более num_shards раз (далее блокируемся). - * Пороги проверяются при вставке, а также, периодически, в фоновом потоке (чтобы реализовать пороги по времени). - * Пороги действуют независимо для каждого shard-а. Каждый shard может быть сброшен независимо от других. - * Если в таблицу вставляется блок, который сам по себе превышает max-пороги, то он записывается сразу в подчинённую таблицу без буферизации. - * Пороги могут быть превышены. Например, если max_rows = 1 000 000, в буфере уже было 500 000 строк, - * и добавляется кусок из 800 000 строк, то в буфере окажется 1 300 000 строк, и затем такой блок будет записан в подчинённую таблицу + * The buffer is a set of num_shards blocks. + * When writing, select the block number by the remainder of the `ThreadNumber` division by `num_shards` (or one of the others), + * and add rows to the corresponding block. + * When using a block, it is blocked by some mutex. If during write the corresponding block is already occupied + * - try to block the next block clockwise, and so no more than `num_shards` times (further blocked). + * Thresholds are checked on insertion, and, periodically, in the background thread (to implement time thresholds). + * Thresholds act independently for each shard. Each shard can be flushed independently of the others. + * If a block is inserted into the table, which itself exceeds the max-thresholds, it is written directly to the subordinate table without buffering. + * Thresholds can be exceeded. For example, if max_rows = 1 000 000, the buffer already had 500 000 rows, + * and a part of 800,000 lines is added, then there will be 1 300 000 rows in the buffer, and then such a block will be written to the subordinate table * - * При уничтожении таблицы типа Buffer и при завершении работы, все данные сбрасываются. - * Данные в буфере не реплицируются, не логгируются на диск, не индексируются. При грубом перезапуске сервера, данные пропадают. + * When you destroy a Buffer type table and when you quit, all data is discarded. + * The data in the buffer is not replicated, not logged to disk, not indexed. With a rough restart of the server, the data is lost. */ class StorageBuffer : private ext::shared_ptr_helper, public IStorage { @@ -43,16 +43,16 @@ friend class BufferBlockInputStream; friend class BufferBlockOutputStream; public: - /// Пороги. + /// Thresholds. struct Thresholds { - time_t time; /// Количество секунд от момента вставки первой строчки в блок. - size_t rows; /// Количество строк в блоке. - size_t bytes; /// Количество (несжатых) байт в блоке. + time_t time; /// The number of seconds from the insertion of the first row into the block. + size_t rows; /// The number of rows in the block. + size_t bytes; /// The number of (uncompressed) bytes in the block. }; - /** num_shards - уровень внутреннего параллелизма (количество независимых буферов) - * Буфер сбрасывается, если превышены все минимальные пороги или хотя бы один из максимальных. + /** num_shards - the level of internal parallelism (the number of independent buffers) + * The buffer is reset if all minimum thresholds or at least one of the maximum thresholds are exceeded. */ static StoragePtr create(const std::string & name_, NamesAndTypesListPtr columns_, const NamesAndTypesList & materialized_columns_, @@ -78,7 +78,7 @@ public: BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; - /// Сбрасывает все буферы в подчинённую таблицу. + /// Resets all buffers to the subordinate table. void shutdown() override; bool optimize(const String & partition, bool final, const Settings & settings) override; @@ -90,7 +90,7 @@ public: bool supportsIndexForIn() const override { return true; } bool supportsParallelReplicas() const override { return true; } - /// Структура подчинённой таблицы не проверяется и не изменяется. + /// The structure of the subordinate table is not checked and does not change. void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override; private: @@ -106,7 +106,7 @@ private: std::mutex mutex; }; - /// Имеется num_shards независимых буферов. + /// There are `num_shards` of independent buffers. const size_t num_shards; std::vector buffers; @@ -115,12 +115,12 @@ private: const String destination_database; const String destination_table; - bool no_destination; /// Если задано - не записывать данные из буфера, а просто опустошать буфер. + bool no_destination; /// If set, do not write data from the buffer, but simply empty the buffer. Poco::Logger * log; Poco::Event shutdown_event; - /// Выполняет сброс данных по таймауту. + /// Resets data by timeout. std::thread flush_thread; StorageBuffer(const std::string & name_, NamesAndTypesListPtr columns_, @@ -132,12 +132,12 @@ private: const String & destination_database_, const String & destination_table_); void flushAllBuffers(bool check_thresholds = true); - /// Сбросить буфер. Если выставлено check_thresholds - сбрасывает только если превышены пороги. + /// Reset the buffer. If check_thresholds is set - resets only if thresholds are exceeded. void flushBuffer(Buffer & buffer, bool check_thresholds); bool checkThresholds(const Buffer & buffer, time_t current_time, size_t additional_rows = 0, size_t additional_bytes = 0) const; bool checkThresholdsImpl(size_t rows, size_t bytes, time_t time_passed) const; - /// Аргумент table передаётся, так как иногда вычисляется заранее. Он должен соответствовать destination-у. + /// `table` argument is passed, as it is sometimes evaluated beforehand. It must match the `destination`. void writeBlockToDestination(const Block & block, StoragePtr table); void flushThread(); diff --git a/dbms/src/Storages/StorageCloud.h b/dbms/src/Storages/StorageCloud.h index 695e182b8b4..e63367699eb 100644 --- a/dbms/src/Storages/StorageCloud.h +++ b/dbms/src/Storages/StorageCloud.h @@ -13,8 +13,8 @@ class DatabaseCloud; class Context; -/** Облачная таблица. Может находиться только в облачной базе данных. - * При записи в таблицу, данные записываются в локальные таблицы на нескольких серверах облака. +/** Cloud table. It can only be in the cloud database. + * When writing to a table, data is written to local tables on multiple cloud servers. */ class StorageCloud : private ext::shared_ptr_helper, public IStorage { @@ -32,7 +32,7 @@ public: std::string getName() const override { return "Cloud"; } std::string getTableName() const override { return name; } - /// Проверка откладывается до метода read. Там проверяется поддержка у использующихся таблиц. + /// The check is postponed to the `read` method. It checks the support of the tables used. bool supportsSampling() const override { return true; } bool supportsPrewhere() const override { return true; } bool supportsFinal() const override { return true; } @@ -50,7 +50,7 @@ public: size_t max_block_size = DEFAULT_BLOCK_SIZE, unsigned threads = 1) override; - void drop() override {} /// Вся нужная работа в DatabaseCloud::removeTable + void drop() override {} /// All the necessary work in `DatabaseCloud::removeTable` void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { diff --git a/dbms/src/Storages/StorageDistributed.h b/dbms/src/Storages/StorageDistributed.h index a73bc80b9b9..db6c3417ef4 100644 --- a/dbms/src/Storages/StorageDistributed.h +++ b/dbms/src/Storages/StorageDistributed.h @@ -18,11 +18,11 @@ class Context; class StorageDistributedDirectoryMonitor; -/** Распределённая таблица, находящаяся на нескольких серверах. - * Использует данные заданной БД и таблицы на каждом сервере. +/** A distributed table that resides on multiple servers. + * Uses data from the specified database and tables on each server. * - * Можно передать один адрес, а не несколько. - * В этом случае, таблицу можно считать удалённой, а не распределённой. + * You can pass one address, not several. + * In this case, the table can be considered remote, rather than distributed. */ class StorageDistributed : private ext::shared_ptr_helper, public IStorage { @@ -32,23 +32,23 @@ class StorageDistributed : private ext::shared_ptr_helper, p public: static StoragePtr create( - const std::string & name_, /// Имя таблицы. - NamesAndTypesListPtr columns_, /// Список столбцов. + const std::string & name_, /// The name of the table. + NamesAndTypesListPtr columns_, /// List of columns. const NamesAndTypesList & materialized_columns_, const NamesAndTypesList & alias_columns_, const ColumnDefaults & column_defaults_, - const String & remote_database_, /// БД на удалённых серверах. - const String & remote_table_, /// Имя таблицы на удалённых серверах. + const String & remote_database_, /// database on remote servers. + const String & remote_table_, /// The name of the table on the remote servers. const String & cluster_name, Context & context_, const ASTPtr & sharding_key_, const String & data_path_); static StoragePtr create( - const std::string & name_, /// Имя таблицы. - NamesAndTypesListPtr columns_, /// Список столбцов. - const String & remote_database_, /// БД на удалённых серверах. - const String & remote_table_, /// Имя таблицы на удалённых серверах. + const std::string & name_, /// The name of the table. + NamesAndTypesListPtr columns_, /// List of columns. + const String & remote_database_, /// database on remote servers. + const String & remote_table_, /// The name of the table on the remote servers. ClusterPtr & owned_cluster_, Context & context_); @@ -78,8 +78,8 @@ public: void drop() override {} void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; } - /// в подтаблицах добавлять и удалять столбы нужно вручную - /// структура подтаблиц не проверяется + /// in the sub-tables, you need to manually add and delete columns + /// the structure of the sub-table is not checked void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override; void shutdown() override; @@ -90,7 +90,7 @@ public: const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, const Settings & settings) override; - /// От каждой реплики получить описание соответствующей локальной таблицы. + /// From each replica, get a description of the corresponding local table. BlockInputStreams describe(const Context & context, const Settings & settings); const ExpressionActionsPtr & getShardingKeyExpr() const { return sharding_key_expr; } @@ -156,7 +156,7 @@ private: bool has_sharding_key; ExpressionActionsPtr sharding_key_expr; String sharding_key_column_name; - String path; /// Может быть пустым, если data_path_ пустой. В этом случае, директория для данных для отправки не создаётся. + String path; /// Can be empty if data_path_ is empty. In this case, a directory for the data to be sent is not created. std::unordered_map> directory_monitors; }; diff --git a/dbms/src/Storages/StorageFactory.h b/dbms/src/Storages/StorageFactory.h index c5b1b5ec43f..e3fbfb4f5ad 100644 --- a/dbms/src/Storages/StorageFactory.h +++ b/dbms/src/Storages/StorageFactory.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Позволяет создать таблицу по имени движка. +/** Allows you to create a table by the name of the engine. */ class StorageFactory : public Singleton { diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index 5397cffa871..0ad21bc2bb3 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -13,12 +13,12 @@ class Join; using JoinPtr = std::shared_ptr; -/** Позволяет сохранить состояние для последующего использования в правой части JOIN. - * При вставке в таблицу, данные будут вставлены в состояние, - * а также записаны в файл-бэкап, для восстановления после перезапуска. - * Чтение из таблицы напрямую невозможно - возможно лишь указание в правой части JOIN. +/** Allows you save the state for later use on the right side of the JOIN. + * When inserted into a table, the data will be inserted into the state, + * and also written to the backup file, to restore after the restart. + * Reading from the table is not possible directly - only specifying on the right side of JOIN is possible. * - * При использовании, JOIN должен быть соответствующего типа (ANY|ALL LEFT|INNER ...). + * When using, JOIN must be of the appropriate type (ANY|ALL LEFT|INNER ...). */ class StorageJoin : private ext::shared_ptr_helper, public StorageSetOrJoinBase { @@ -43,10 +43,10 @@ public: String getName() const override { return "Join"; } - /// Получить доступ к внутренностям. + /// Access the innards. JoinPtr & getJoin() { return join; } - /// Убедиться, что структура данных подходит для осуществления JOIN такого типа. + /// Verify that the data structure is suitable for implementing this type of JOIN. void assertCompatible(ASTTableJoin::Kind kind_, ASTTableJoin::Strictness strictness_) const; private: diff --git a/dbms/src/Storages/StorageLog.h b/dbms/src/Storages/StorageLog.h index 33eeb44d068..0479230a583 100644 --- a/dbms/src/Storages/StorageLog.h +++ b/dbms/src/Storages/StorageLog.h @@ -21,22 +21,22 @@ namespace ErrorCodes } -/** Смещение до каждой некоторой пачки значений. - * Эти пачки имеют одинаковый размер в разных столбцах. - * Они нужны, чтобы можно было читать данные в несколько потоков. +/** Offsets to every single set of values. + * These sets are the same size in different columns. + * They are needed so that you can read the data in several threads. */ struct Mark { - size_t rows; /// Сколько строк содержится в этой пачке и всех предыдущих. - size_t offset; /// Смещение до пачки в сжатом файле. + size_t rows; /// How many lines are contained in this set and all previous ones. + size_t offset; /// The offset to the set in the compressed file. }; using Marks = std::vector; -/** Реализует хранилище, подходящее для логов. - * Ключи не поддерживаются. - * Данные хранятся в сжатом виде. +/** Implements a repository that is suitable for logs. + * Keys are not supported. + * The data is stored in a compressed form. */ class StorageLog : private ext::shared_ptr_helper, public IStorage { @@ -45,9 +45,9 @@ friend class LogBlockInputStream; friend class LogBlockOutputStream; public: - /** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце), - * (корректность имён и путей не проверяется) - * состоящую из указанных столбцов; создать файлы, если их нет. + /** hook the table with the appropriate name, along the appropriate path (with / at the end), + * (the correctness of names and paths is not verified) + * consisting of the specified columns; Create files if they do not exist. */ static StoragePtr create( const std::string & path_, @@ -82,11 +82,11 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; - /// Данные столбца + /// Column data struct ColumnData { - /// Задает номер столбца в файле с засечками. - /// Не обязательно совпадает с номером столбца среди столбцов таблицы: здесь нумеруются также столбцы с длинами массивов. + /// Specifies the column number in the marks file. + /// Does not necessarily match the column number among the columns of the table: columns with lengths of arrays are also numbered here. size_t column_index; Poco::File data_file; @@ -112,12 +112,12 @@ protected: const ColumnDefaults & column_defaults_, size_t max_compress_block_size_); - /// Прочитать файлы с засечками, если они ещё не прочитаны. - /// Делается лениво, чтобы при большом количестве таблиц, сервер быстро стартовал. - /// Нельзя вызывать с залоченным на запись rwlock. + /// Read marks files if they are not already read. + /// It is done lazily, so that with a large number of tables, the server starts quickly. + /// You can not call with a write locked `rwlock`. void loadMarks(); - /// Можно вызывать при любом состоянии rwlock. + /// Can be called with any state of `rwlock`. size_t marksCount(); BlockInputStreams read( @@ -143,7 +143,7 @@ private: void loadMarksImpl(bool load_null_marks); - /// Порядок добавления файлов не должен меняться: он соответствует порядку столбцов в файле с засечками. + /// The order of adding files should not change: it corresponds to the order of the columns in the marks file. void addFile(const String & column_name, const IDataType & type, size_t level = 0); bool loaded_marks; @@ -157,12 +157,12 @@ protected: FileChecker file_checker; private: - /** Для обычных столбцов, в засечках указано количество строчек в блоке. - * Для столбцов-массивов и вложенных структур, есть более одной группы засечек, соответствующих разным файлам: - * - для внутренностей (файла name.bin) - указано суммарное количество элементов массивов в блоке, - * - для размеров массивов (файла name.size0.bin) - указано количество строчек (самих целых массивов) в блоке. + /** For normal columns, the number of rows in the block is specified in the marks. + * For array columns and nested structures, there are more than one group of marks that correspond to different files + * - for insides (file name.bin) - the total number of array elements in the block is specified, + * - for array sizes (file name.size0.bin) - the number of rows (the whole arrays themselves) in the block is specified. * - * Вернуть первую попавшуюся группу засечек, в которых указано количество строчек, а не внутренностей массивов. + * Return the first group of marks that contain the number of rows, but not the internals of the arrays. */ const Marks & getMarksWithRealRowCount() const; diff --git a/dbms/src/Storages/StorageMemory.h b/dbms/src/Storages/StorageMemory.h index 6bfb256d8f2..e75dde8aec1 100644 --- a/dbms/src/Storages/StorageMemory.h +++ b/dbms/src/Storages/StorageMemory.h @@ -15,10 +15,10 @@ namespace DB class StorageMemory; -/** Реализует хранилище в оперативке. - * Подходит для временных данных. - * В нём не поддерживаются ключи. - * Данные хранятся в виде набора блоков и никуда дополнительно не сохраняются. +/** Implements storage in the RAM. + * Suitable for temporary data. + * It does not support keys. + * Data is stored as a set of blocks and is not stored anywhere else. */ class StorageMemory : private ext::shared_ptr_helper, public IStorage { @@ -63,7 +63,7 @@ private: String name; NamesAndTypesListPtr columns; - /// Сами данные. list - чтобы при вставке в конец, существующие итераторы не инвалидировались. + /// The data itself. `list` - so that when inserted to the end, the existing iterators are not invalidated. BlocksList data; std::mutex mutex; diff --git a/dbms/src/Storages/StorageMerge.h b/dbms/src/Storages/StorageMerge.h index b6798df31b4..466831e9bc1 100644 --- a/dbms/src/Storages/StorageMerge.h +++ b/dbms/src/Storages/StorageMerge.h @@ -9,8 +9,8 @@ namespace DB { -/** Таблица, представляющая собой объединение произвольного количества других таблиц. - * У всех таблиц должна быть одинаковая структура. +/** A table that represents the union of an arbitrary number of other tables. + * All tables must have the same structure. */ class StorageMerge : private ext::shared_ptr_helper, public IStorage { @@ -18,26 +18,26 @@ friend class ext::shared_ptr_helper; public: static StoragePtr create( - const std::string & name_, /// Имя таблицы. - NamesAndTypesListPtr columns_, /// Список столбцов. - const String & source_database_, /// В какой БД искать таблицы-источники. - const String & table_name_regexp_, /// Регексп имён таблиц-источников. - const Context & context_); /// Известные таблицы. + const std::string & name_, /// The name of the table. + NamesAndTypesListPtr columns_, /// List of columns. + const String & source_database_, /// In which database to look for source tables. + const String & table_name_regexp_, /// Regex names of source tables. + const Context & context_); /// Known tables. static StoragePtr create( - const std::string & name_, /// Имя таблицы. - NamesAndTypesListPtr columns_, /// Список столбцов. + const std::string & name_, /// The name of the table. + NamesAndTypesListPtr columns_, /// List of columns. const NamesAndTypesList & materialized_columns_, const NamesAndTypesList & alias_columns_, const ColumnDefaults & column_defaults_, - const String & source_database_, /// В какой БД искать таблицы-источники. - const String & table_name_regexp_, /// Регексп имён таблиц-источников. - const Context & context_); /// Известные таблицы. + const String & source_database_, /// In which database to look for source tables. + const String & table_name_regexp_, /// Regex names of source tables. + const Context & context_); /// Known tables. std::string getName() const override { return "Merge"; } std::string getTableName() const override { return name; } - /// Проверка откладывается до метода read. Там проверяется поддержка у использующихся таблиц. + /// The check is delayed to the read method. It checks the support of the tables used. bool supportsSampling() const override { return true; } bool supportsPrewhere() const override { return true; } bool supportsFinal() const override { return true; } @@ -59,8 +59,8 @@ public: void drop() override {} void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; } - /// в подтаблицах добавлять и удалять столбы нужно вручную - /// структура подтаблиц не проверяется + /// you need to add and remove columns in the sub-tables manually + /// the structure of sub-tables is not checked void alter(const AlterCommands & params, const String & database_name, const String & table_name, const Context & context) override; private: diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index 24655c58b24..6b10505dc9b 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -533,7 +533,7 @@ void StorageMergeTree::attachPartition(ASTPtr query, const Field & field, bool u LOG_INFO(log, "Finished attaching part"); } - /// New parts with other data may appear in place of deleted pieces. + /// New parts with other data may appear in place of deleted parts. context.resetCaches(); } diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index d60c6477c8d..7276d5edf22 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -14,7 +14,7 @@ namespace DB { -/** См. описание структуры данных в MergeTreeData. +/** See the description of the data structure in MergeTreeData. */ class StorageMergeTree : private ext::shared_ptr_helper, public IStorage { @@ -22,13 +22,13 @@ friend class ext::shared_ptr_helper; friend class MergeTreeBlockOutputStream; public: - /** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце), - * (корректность имён и путей не проверяется) - * состоящую из указанных столбцов. + /** hook the table with the appropriate name, along the appropriate path (with / at the end), + * (correctness of names and paths are not checked) + * consisting of the specified columns. * - * primary_expr_ast - выражение для сортировки; - * date_column_name - имя столбца с датой; - * index_granularity - на сколько строчек пишется одно значение индекса. + * primary_expr_ast - expression for sorting; + * date_column_name - the name of the column with the date; + * index_granularity - fow how many rows one index value is written. */ static StoragePtr create( const String & path_, @@ -42,7 +42,7 @@ public: Context & context_, ASTPtr & primary_expr_ast_, const String & date_column_name_, - const ASTPtr & sampling_expression_, /// nullptr, если семплирование не поддерживается. + const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. size_t index_granularity_, const MergeTreeData::MergingParams & merging_params_, bool has_force_restore_data_flag, @@ -85,7 +85,7 @@ public: BlockOutputStreamPtr write(ASTPtr query, const Settings & settings) override; - /** Выполнить очередной шаг объединения кусков. + /** Perform the next step in combining the parts. */ bool optimize(const String & partition, bool final, const Settings & settings) override { diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index d7c1efaab83..b11405b6434 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -11,8 +11,8 @@ namespace DB { -/** При записи, ничего не делает. - * При чтении, возвращает пустоту. +/** When writing, does nothing. + * When reading, returns nothing. */ class StorageNull : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index ee6483a9139..c4055c2266b 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -127,8 +127,8 @@ static const auto MERGE_SELECTING_SLEEP_MS = 5 * 1000; * Why is this number 200? * The fact is that previously negative block numbers were not supported. * And also, the merge is done that way so that when you increase the number of parts, insertion of new parts slows down on purpose, - * until mergers have time to reduce the number of parts; and it was calculated for about 200 pieces. - * So, when you insert all the parts from the other table into the table, 200 numbers are sure enough. + * until mergers have time to reduce the number of parts; and it was calculated for about 200 parts. + * So, when you insert all the parts from the other table into the table, 200 is sure enough. * In turn, this number is chosen almost at random. */ extern const Int64 RESERVED_BLOCK_NUMBERS = 200; @@ -2803,7 +2803,7 @@ void StorageReplicatedMergeTree::dropPartition( String fake_part_name = getFakePartNameForDrop(month_name, left, right); /** Forbid to choose the parts to be deleted for merging. - * Invariant: after the `DROP_RANGE` entry appears in the log, merge of deleted pieces will not appear in the log. + * Invariant: after the `DROP_RANGE` entry appears in the log, merge of deleted parts will not appear in the log. */ { std::lock_guard merge_selecting_lock(merge_selecting_mutex); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index 93c3dc64f49..2a99ce4a46e 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -28,45 +28,45 @@ namespace DB { -/** Движок, использующий merge-дерево (см. MergeTreeData) и реплицируемый через ZooKeeper. +/** The engine that uses the merge tree (see MergeTreeData) and replicated through ZooKeeper. * - * ZooKeeper используется для следующих вещей: - * - структура таблицы (/metadata, /columns) - * - лог действий с данными (/log/log-..., /replicas/replica_name/queue/queue-...); - * - список реплик (/replicas), признак активности реплики (/replicas/replica_name/is_active), адреса реплик (/replicas/replica_name/host); - * - выбор реплики-лидера (/leader_election) - это та реплика, которая назначает мерджи; - * - набор кусков данных на каждой реплике (/replicas/replica_name/parts); - * - список последних N блоков данных с чексуммами, для дедупликации (/blocks); - * - список инкрементальных номеров блоков (/block_numbers), которые мы сейчас собираемся вставить, - * или которые были неиспользованы (/nonincremental_block_numbers) - * для обеспечения линейного порядка вставки данных и мерджа данных только по интервалам в этой последовательности; - * - координация записей с кворумом (/quorum). + * ZooKeeper is used for the following things: + * - the structure of the table (/ metadata, /columns) + * - action log with data (/log/log-...,/replicas/replica_name/queue/queue-...); + * - a replica list (/replicas), and replica activity tag (/replicas/replica_name/is_active), replica addresses (/replicas/replica_name/host); + * - select the leader replica (/leader_election) - this is the replica that assigns the merge; + * - a set of parts of data on each replica (/replicas/replica_name/parts); + * - list of the last N blocks of data with checksum, for deduplication (/blocks); + * - the list of incremental block numbers (/block_numbers) that we are about to insert, + * or that were unused (/nonincremental_block_numbers) + * to ensure the linear order of data insertion and data merge only on the intervals in this sequence; + * - coordinates writes with quorum (/quorum). */ -/** У реплицируемых таблиц есть общий лог (/log/log-...). - * Лог - последовательность записей (LogEntry) о том, что делать. - * Каждая запись - это одно из: - * - обычная вставка данных (GET), - * - мердж (MERGE), - * - чуть менее обычная вставка данных (ATTACH), - * - удаление партиции (DROP). +/** The replicated tables have a common log (/log/log-...). + * Log - a sequence of entries (LogEntry) about what to do. + * Each entry is one of: + * - normal data insertion (GET), + * - merge (MERGE), + * - slightly less common data insertion (ATTACH), + * - delete the partition (DROP). * - * Каждая реплика копирует (queueUpdatingThread, pullLogsToQueue) записи из лога в свою очередь (/replicas/replica_name/queue/queue-...), - * и затем выполняет их (queueTask). - * Не смотря на название "очередь", выполнение может переупорядочиваться, при необходимости (shouldExecuteLogEntry, executeLogEntry). - * Кроме того, записи в очереди могут генерироваться самостоятельно (не из лога), в следующих случаях: - * - при создании новой реплики, туда помещаются действия на GET с других реплик (createReplica); - * - если кусок повреждён (removePartAndEnqueueFetch) или отсутствовал при проверке (при старте - checkParts, во время работы - searchForMissingPart), - * туда помещаются действия на GET с других реплик; + * Each replica copies (queueUpdatingThread, pullLogsToQueue) entries from the log to its queue (/replicas/replica_name/queue/queue-...) + * and then executes them (queueTask). + * Despite the name of the "queue", execution can be reordered, if necessary (shouldExecuteLogEntry, executeLogEntry). + * In addition, the records in the queue can be generated independently (not from the log), in the following cases: + * - when creating a new replica, actions are put on GET from other replicas (createReplica); + * - if the part is corrupt (removePartAndEnqueueFetch) or absent during the check (at start - checkParts, while running - searchForMissingPart), + * actions are put on GET from other replicas; * - * У реплики, на которую был сделан INSERT, в очереди тоже будет запись о GET этих данных. - * Такая запись считается выполненной, как только обработчик очереди её увидит. + * The replica to which INSERT was made in the queue will also have an entry of the GET of this data. + * Such an entry is considered to be executed as soon as the queue handler sees it. * - * У записи в логе есть время создания. Это время генерируется по часам на сервере, который создал запись - * - того, на которых пришёл соответствующий запрос INSERT или ALTER. + * The log entry has a creation time. This time is generated by the clock of server that created entry + * - the one on which the corresponding INSERT or ALTER query came. * - * Для записей в очереди, которые реплика сделала для себя самостоятельно, - * в качестве времени будет браться время создания соответствующего куска на какой-либо из реплик. + * For the entries in the queue that the replica made for itself, + * as the time will take the time of creation the appropriate part on any of the replicas. */ class StorageReplicatedMergeTree : private ext::shared_ptr_helper, public IStorage @@ -74,7 +74,7 @@ class StorageReplicatedMergeTree : private ext::shared_ptr_helper; public: - /** Если !attach, либо создает новую таблицу в ZK, либо добавляет реплику в существующую таблицу. + /** If !attach, either creates a new table in ZK, or adds a replica to an existing table. */ static StoragePtr create( const String & zookeeper_path_, @@ -88,7 +88,7 @@ public: Context & context_, ASTPtr & primary_expr_ast_, const String & date_column_name_, - const ASTPtr & sampling_expression_, /// nullptr, если семплирование не поддерживается. + const ASTPtr & sampling_expression_, /// nullptr, if sampling is not supported. size_t index_granularity_, const MergeTreeData::MergingParams & merging_params_, bool has_force_restore_data_flag, @@ -148,7 +148,7 @@ public: const ASTPtr & sharding_key_expr, bool do_copy, const Field & coordinator, const Settings & settings) override; - /** Удаляет реплику из ZooKeeper. Если других реплик нет, удаляет всю таблицу из ZooKeeper. + /** Removes a replica from ZooKeeper. If there are no other replicas, it deletes the entire table from ZooKeeper. */ void drop() override; @@ -163,7 +163,7 @@ public: MergeTreeData * getUnreplicatedData() { return unreplicated_data.get(); } - /** Для системной таблицы replicas. */ + /** For the system table replicas. */ struct Status { bool is_leader; @@ -181,7 +181,7 @@ public: UInt8 active_replicas; }; - /// Получить статус таблицы. Если with_zk_fields = false - не заполнять поля, требующие запросов в ZK. + /// Get the status of the table. If with_zk_fields = false - do not fill in the fields that require queries to ZK. void getStatus(Status & res, bool with_zk_fields = true); using LogEntriesData = std::vector; @@ -189,7 +189,7 @@ public: void getReplicaDelays(time_t & out_absolute_delay, time_t & out_relative_delay); - /// Добавить кусок в очередь кусков, чьи данные нужно проверить в фоновом потоке. + /// Add a part to the queue of parts whose data you want to check in the background thread. void enqueuePartForCheck(const String & part_name, time_t delay_to_check_seconds = 0) { part_check_thread.enqueuePart(part_name, delay_to_check_seconds); @@ -216,14 +216,14 @@ private: Context & context; - zkutil::ZooKeeperPtr current_zookeeper; /// Используйте только с помощью методов ниже. - std::mutex current_zookeeper_mutex; /// Для пересоздания сессии в фоновом потоке. + zkutil::ZooKeeperPtr current_zookeeper; /// Use only the methods below. + std::mutex current_zookeeper_mutex; /// To recreate the session in the background thread. zkutil::ZooKeeperPtr tryGetZooKeeper(); zkutil::ZooKeeperPtr getZooKeeper(); void setZooKeeper(zkutil::ZooKeeperPtr zookeeper); - /// Если true, таблица в офлайновом режиме, и в нее нельзя писать. + /// If true, the table is offline and can not be written to it. bool is_readonly = false; String database_name; @@ -234,8 +234,8 @@ private: String replica_name; String replica_path; - /** Очередь того, что нужно сделать на этой реплике, чтобы всех догнать. Берется из ZooKeeper (/replicas/me/queue/). - * В ZK записи в хронологическом порядке. Здесь - не обязательно. + /** The queue of what needs to be done on this replica to catch up with everyone. It is taken from ZooKeeper (/replicas/me/queue/). + * In ZK entries in chronological order. Here it is not necessary. */ ReplicatedMergeTreeQueue queue; @@ -243,12 +243,12 @@ private: */ zkutil::EphemeralNodeHolderPtr replica_is_active_node; - /** Версия ноды /columns в ZooKeeper, соответствующая текущим data.columns. - * Читать и изменять вместе с data.columns - под TableStructureLock. + /** Version node /columns in ZooKeeper corresponding to the current data.columns. + * Read and modify along with the data.columns - under TableStructureLock. */ int columns_version = -1; - /** Является ли эта реплика "ведущей". Ведущая реплика выбирает куски для слияния. + /** Is this replica "master". The master replica selects the parts to merge. */ bool is_leader_node = false; std::mutex leader_node_mutex; @@ -272,46 +272,46 @@ private: zkutil::LeaderElectionPtr leader_election; - /// Для чтения данных из директории unreplicated. + /// To read data from the `unreplicated` directory. std::unique_ptr unreplicated_data; std::unique_ptr unreplicated_reader; std::unique_ptr unreplicated_merger; - std::mutex unreplicated_mutex; /// Для мерджей и удаления нереплицируемых кусков. + std::mutex unreplicated_mutex; /// For merge and removal of non-replicable parts. - /// Нужно ли завершить фоновые потоки (кроме restarting_thread). + /// Do I need to complete background threads (except restarting_thread)? std::atomic shutdown_called {false}; Poco::Event shutdown_event; /// Limiting parallel fetches per one table std::atomic_uint current_table_fetches {0}; - /// Потоки: + /// Streams - /// Поток, следящий за обновлениями в логах всех реплик и загружающий их в очередь. + /// A thread that keeps track of the updates in the logs of all replicas and loads them into the queue. std::thread queue_updating_thread; zkutil::EventPtr queue_updating_event = std::make_shared(); - /// Задание, выполняющее действия из очереди. + /// A task that performs actions from the queue. BackgroundProcessingPool::TaskHandle queue_task_handle; - /// Поток, выбирающий куски для слияния. + /// A thread that selects parts to merge. std::thread merge_selecting_thread; Poco::Event merge_selecting_event; - std::mutex merge_selecting_mutex; /// Берется на каждую итерацию выбора кусков для слияния. + std::mutex merge_selecting_mutex; /// It is taken for each iteration of the selection of parts to merge. - /// Поток, удаляющий старые куски, записи в логе и блоки. + /// A thread that removes old parts, log entries, and blocks. std::unique_ptr cleanup_thread; - /// Поток, обрабатывающий переподключение к ZooKeeper при истечении сессии. + /// A thread that processes reconnection to ZooKeeper when the session expires. std::unique_ptr restarting_thread; - /// Поток, следящий за изменениями списка столбцов в ZooKeeper и обновляющий куски в соответствии с этими изменениями. + /// A thread monitoring changes to the column list in ZooKeeper and updating the parts in accordance with these changes. std::unique_ptr alter_thread; - /// Поток, проверяющий данные кусков, а также очередь кусков для проверки. + /// A thread that checks the data of the parts, as well as the queue of the parts to be checked. ReplicatedMergeTreePartCheckThread part_check_thread; - /// Событие, пробуждающее метод alter от ожидания завершения запроса ALTER. + /// An event that awakens `alter` method from waiting for the completion of the ALTER query. zkutil::EventPtr alter_query_event = std::make_shared(); Logger * log; @@ -334,102 +334,102 @@ private: bool has_force_restore_data_flag, const MergeTreeSettings & settings_); - /// Инициализация. + /// Initialization. - /** Создает минимальный набор нод в ZooKeeper. + /** Creates the minimum set of nodes in ZooKeeper. */ void createTableIfNotExists(); - /** Создает реплику в ZooKeeper и добавляет в очередь все, что нужно, чтобы догнать остальные реплики. + /** Creates a replica in ZooKeeper and adds to the queue all that it takes to catch up with the rest of the replicas. */ void createReplica(); - /** Создать узлы в ZK, которые должны быть всегда, но которые могли не существовать при работе старых версий сервера. + /** Create nodes in the ZK, which must always be, but which might not exist when older versions of the server are running. */ void createNewZooKeeperNodes(); - /** Проверить, что список столбцов и настройки таблицы совпадают с указанными в ZK (/metadata). - * Если нет - бросить исключение. + /** Verify that the list of columns and table settings match those specified in ZK (/metadata). + * If not, throw an exception. */ void checkTableStructure(bool skip_sanity_checks, bool allow_alter); - /** Проверить, что множество кусков соответствует тому, что в ZK (/replicas/me/parts/). - * Если каких-то кусков, описанных в ZK нет локально, бросить исключение. - * Если какие-то локальные куски не упоминаются в ZK, удалить их. - * Но если таких слишком много, на всякий случай бросить исключение - скорее всего, это ошибка конфигурации. + /** Check that the set of parts corresponds to that in ZK (/replicas/me/parts/). + * If any parts described in ZK are not locally, throw an exception. + * If any local parts are not mentioned in ZK, remove them. + * But if there are too many, throw an exception just in case - it's probably a configuration error. */ void checkParts(bool skip_sanity_checks); - /** Проверить, что чексумма куска совпадает с чексуммой того же куска на какой-нибудь другой реплике. - * Если ни у кого нет такого куска, ничего не проверяет. - * Не очень надежно: если две реплики добавляют кусок почти одновременно, ни одной проверки не произойдет. - * Кладет в ops действия, добавляющие данные о куске в ZooKeeper. - * Вызывать под TableStructureLock. + /** Check that the part's checksum is the same as the checksum of the same part on some other replica. + * If no one has such a part, nothing checks. + * Not very reliable: if two replicas add a part almost at the same time, no checks will occur. + * Adds actions to `ops` that add data about the part into ZooKeeper. + * Call under TableStructureLock. */ void checkPartAndAddToZooKeeper(const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String name_override = ""); - /** Исходит из допущения, что такого куска ещё нигде нет (Это обеспечено, если номер куска выделен с помощью AbandonableLock). - * Кладет в ops действия, добавляющие данные о куске в ZooKeeper. - * Вызывать под TableStructureLock. + /** Based on the assumption that there is no such part anywhere else (This is provided if the part number is highlighted with AbandonableLock). + * Adds actions to `ops` that add data about the part into ZooKeeper. + * Call under TableStructureLock. */ void addNewPartToZooKeeper(const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String name_override = ""); - /// Кладет в ops действия, удаляющие кусок из ZooKeeper. + /// Adds actions to `ops` that remove a part from ZooKeeper. void removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops); - /// Убирает кусок из ZooKeeper и добавляет в очередь задание скачать его. Предполагается это делать с битыми кусками. + /// Removes a part from ZooKeeper and adds a task to the queue to download it. It is supposed to do this with broken parts. void removePartAndEnqueueFetch(const String & part_name); - /// Выполнение заданий из очереди. + /// Running jobs from the queue. - /** Копирует новые записи из логов всех реплик в очередь этой реплики. - * Если next_update_event != nullptr, вызовет это событие, когда в логе появятся новые записи. + /** Copies the new entries from the logs of all replicas to the queue of this replica. + * If next_update_event != nullptr, calls this event when new entries appear in the log. */ void pullLogsToQueue(zkutil::EventPtr next_update_event = nullptr); - /** Выполнить действие из очереди. Бросает исключение, если что-то не так. - * Возвращает, получилось ли выполнить. Если не получилось, запись нужно положить в конец очереди. + /** Execute the action from the queue. Throws an exception if something is wrong. + * Returns whether or not it succeeds. If it did not work, write it to the end of the queue. */ bool executeLogEntry(const LogEntry & entry); void executeDropRange(const LogEntry & entry); - bool executeAttachPart(const LogEntry & entry); /// Возвращает false, если куска нет, и его нужно забрать с другой реплики. + bool executeAttachPart(const LogEntry & entry); /// Returns false if the part is absent, and it needs to be picked up from another replica. - /** Обновляет очередь. + /** Updates the queue. */ void queueUpdatingThread(); - /** Выполняет действия из очереди. + /** Performs actions from the queue. */ bool queueTask(); - /// Выбор кусков для слияния. + /// Select the parts to merge. void becomeLeader(); - /** Выбирает куски для слияния и записывает в лог. + /** Selects the parts to merge and writes to the log. */ void mergeSelectingThread(); using MemoizedPartsThatCouldBeMerged = std::set>; - /// Можно ли мерджить куски в указанном диапазоне? memo - необязательный параметр. + /// Is it possible to merge parts in the specified range? `memo` is an optional parameter. bool canMergeParts( const MergeTreeData::DataPartPtr & left, const MergeTreeData::DataPartPtr & right, MemoizedPartsThatCouldBeMerged * memo); - /** Записать выбранные куски для слияния в лог, - * Вызывать при заблокированном merge_selecting_mutex. - * Возвращает false, если какого-то куска нет в ZK. + /** Write the selected parts to merge into the log, + * Call when merge_selecting_mutex is locked. + * Returns false if any part is not in ZK. */ bool createLogEntryToMergeParts( const MergeTreeData::DataPartsVector & parts, const String & merged_name, ReplicatedMergeTreeLogEntryData * out_log_entry = nullptr); - /// Обмен кусками. + /// Exchange parts. - /** Возвращает пустую строку, если куска ни у кого нет. + /** Returns an empty string if no one has a part. */ String findReplicaHavingPart(const String & part_name, bool active); @@ -440,9 +440,9 @@ private: */ String findReplicaHavingCoveringPart(const String & part_name, bool active, String & out_covering_part_name); - /** Скачать указанный кусок с указанной реплики. - * Если to_detached, то кусок помещается в директорию detached. - * Если quorum != 0, то обновляется узел для отслеживания кворума. + /** Download the specified part from the specified replica. + * If `to_detached`, the part is placed in the `detached` directory. + * If quorum != 0, then the node for tracking the quorum is updated. * Returns false if part is already fetching right now. */ bool fetchPart(const String & part_name, const String & replica_path, bool to_detached, size_t quorum); @@ -450,42 +450,42 @@ private: std::unordered_set currently_fetching_parts; std::mutex currently_fetching_parts_mutex; - /** При отслеживаемом кворуме - добавить реплику в кворум для куска. + /** With the quorum being tracked, add a replica to the quorum for the part. */ void updateQuorum(const String & part_name); AbandonableLockInZooKeeper allocateBlockNumber(const String & month_name); - /** Дождаться, пока все реплики, включая эту, выполнят указанное действие из лога. - * Если одновременно с этим добавляются реплики, может не дождаться добавленную реплику. + /** Wait until all replicas, including this, execute the specified action from the log. + * If replicas are added at the same time, it can not wait the added replica . */ void waitForAllReplicasToProcessLogEntry(const ReplicatedMergeTreeLogEntryData & entry); - /** Дождаться, пока указанная реплика выполнит указанное действие из лога. + /** Wait until the specified replica executes the specified action from the log. */ void waitForReplicaToProcessLogEntry(const String & replica_name, const ReplicatedMergeTreeLogEntryData & entry); - /// Кинуть исключение, если таблица readonly. + /// Throw an exception if the table is readonly. void assertNotReadonly() const; - /** Получить блокировку, которая защищает заданную партицию от задачи слияния. - * Блокировка является рекурсивной. + /** Get a lock that protects the specified partition from the merge task. + * The lock is recursive. */ std::string acquirePartitionMergeLock(const std::string & partition_name); - /** Заявить, что больше не ссылаемся на блокировку соответствующую заданной - * партиции. Если ссылок больше нет, блокировка уничтожается. + /** Declare that we no longer refer to the lock corresponding to the specified + * partition. If there are no more links, the lock is destroyed. */ void releasePartitionMergeLock(const std::string & partition_name); - /// Проверить наличие узла в ZK. Если он есть - запомнить эту информацию, и затем сразу отвечать true. + /// Check for a node in ZK. If it is, remember this information, and then immediately answer true. std::unordered_set existing_nodes_cache; std::mutex existing_nodes_cache_mutex; bool existsNodeCached(const std::string & path); - /// Перешардирование. + /// Resharding. struct ReplicaSpaceInfo { long double factor = 0.0; @@ -494,16 +494,16 @@ private: using ReplicaToSpaceInfo = std::map; - /** Проверяет, что структуры локальной и реплицируемых таблиц совпадают. + /** Checks that the structures of the local and replicated tables are the same. */ void enforceShardsConsistency(const WeightedZooKeeperPaths & weighted_zookeeper_paths); - /** Получить информацию о свободном месте на репликах + дополнительную информацию - * для функции checkSpaceForResharding. + /** Get information about free space on replicas + additional information + * for the function checkSpaceForResharding. */ ReplicaToSpaceInfo gatherReplicaSpaceInfo(const WeightedZooKeeperPaths & weighted_zookeeper_paths); - /** Проверяет, что имеется достаточно свободного места локально и на всех репликах. + /** Checks that there is enough free space locally and on all replicas. */ bool checkSpaceForResharding(const ReplicaToSpaceInfo & replica_to_space_info, size_t partition_size) const; }; diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 30baf42f050..1732044bccf 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -12,7 +12,7 @@ class Set; using SetPtr = std::shared_ptr; -/** Общая часть StorageSet и StorageJoin. +/** Common part of StorageSet and StorageJoin. */ class StorageSetOrJoinBase : private ext::shared_ptr_helper, public IStorage { @@ -40,24 +40,24 @@ protected: String name; NamesAndTypesListPtr columns; - UInt64 increment = 0; /// Для имён файлов бэкапа. + UInt64 increment = 0; /// For the backup file names. - /// Восстановление из бэкапа. + /// Restore from backup. void restore(); private: void restoreFromFile(const String & file_path); - /// Вставить блок в состояние. + /// Insert the block into the state. virtual void insertBlock(const Block & block) = 0; virtual size_t getSize() const = 0; }; -/** Позволяет сохранить множество для последующего использования в правой части оператора IN. - * При вставке в таблицу, данные будут вставлены в множество, - * а также записаны в файл-бэкап, для восстановления после перезапуска. - * Чтение из таблицы напрямую невозможно - возможно лишь указание в правой части оператора IN. +/** Lets you save the set for later use on the right side of the IN statement. + * When inserted into a table, the data will be inserted into the set, + * and also written to a file-backup, for recovery after a restart. + * Reading from the table is not possible directly - it is possible to specify only the right part of the IN statement. */ class StorageSet : private ext::shared_ptr_helper, public StorageSetOrJoinBase { @@ -77,7 +77,7 @@ public: String getName() const override { return "Set"; } - /// Получить доступ к внутренностям. + /// Access the insides. SetPtr & getSet() { return set; } private: diff --git a/dbms/src/Storages/StorageStripeLog.h b/dbms/src/Storages/StorageStripeLog.h index 39ca53bd2d5..4265ab36abe 100644 --- a/dbms/src/Storages/StorageStripeLog.h +++ b/dbms/src/Storages/StorageStripeLog.h @@ -14,8 +14,8 @@ namespace DB { -/** Реализует хранилище, подходящее для маленьких кусочков лога. - * При этом, хранит все столбцы в одном файле формата Native, с расположенным рядом индексом. +/** Implements a repository that is suitable for small pieces of the log. + * In doing so, stores all the columns in a single Native file, with a nearby index. */ class StorageStripeLog : private ext::shared_ptr_helper, public IStorage { @@ -24,10 +24,10 @@ friend class StripeLogBlockInputStream; friend class StripeLogBlockOutputStream; public: - /** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце), - * (корректность имён и путей не проверяется) - * состоящую из указанных столбцов. - * Если не указано attach - создать директорию, если её нет. + /** hook the table with the appropriate name, along the appropriate path (with / at the end), + * (the correctness of names and paths is not checked) + * consisting of the specified columns. + * If not specified `attach` - create a directory if it does not exist. */ static StoragePtr create( const std::string & path_, @@ -59,7 +59,7 @@ public: bool checkData() const override; - /// Данные файла. + /// Data of the file. struct ColumnData { Poco::File data_file; diff --git a/dbms/src/Storages/StorageTinyLog.h b/dbms/src/Storages/StorageTinyLog.h index f012e282160..842bce6ef2e 100644 --- a/dbms/src/Storages/StorageTinyLog.h +++ b/dbms/src/Storages/StorageTinyLog.h @@ -14,8 +14,8 @@ namespace DB { -/** Реализует хранилище, подходящее для маленьких кусочков лога. - * Отличается от StorageLog отсутствием файлов с засечками. +/** Implements a repository that is suitable for small pieces of the log. + * It differs from StorageLog in the absence of mark files. */ class StorageTinyLog : private ext::shared_ptr_helper, public IStorage { @@ -24,10 +24,10 @@ friend class TinyLogBlockInputStream; friend class TinyLogBlockOutputStream; public: - /** Подцепить таблицу с соответствующим именем, по соответствующему пути (с / на конце), - * (корректность имён и путей не проверяется) - * состоящую из указанных столбцов. - * Если не указано attach - создать директорию, если её нет. + /** hook the table with the appropriate name, along the appropriate path (with / at the end), + * (the correctness of names and paths is not verified) + * consisting of the specified columns. + * If not specified `attach` - create a directory if it does not exist. */ static StoragePtr create( const std::string & path_, @@ -61,7 +61,7 @@ public: bool checkData() const override; - /// Данные столбца + /// Column data struct ColumnData { Poco::File data_file; diff --git a/dbms/src/Storages/StorageView.h b/dbms/src/Storages/StorageView.h index a6be51cdad5..f36b3b2bff8 100644 --- a/dbms/src/Storages/StorageView.h +++ b/dbms/src/Storages/StorageView.h @@ -29,7 +29,7 @@ public: const NamesAndTypesList & getColumnsListImpl() const override { return *columns; } ASTPtr getInnerQuery() const { return inner_query.clone(); }; - /// Пробрасывается внутрь запроса и решается на его уровне. + /// It is passed inside the query and solved at its level. bool supportsSampling() const override { return true; } bool supportsFinal() const override { return true; } @@ -66,7 +66,7 @@ protected: const ColumnDefaults & column_defaults_); private: - /// Достать из самого внутреннего подзапроса имя базы данных и таблицы: select_database_name, select_table_name. + /// extract the name of the database and the table from the most internal subquery: `select_database_name, select_table_name`. void extractDependentTable(const ASTSelectQuery & query); }; diff --git a/dbms/src/Storages/System/StorageSystemDatabases.h b/dbms/src/Storages/System/StorageSystemDatabases.h index e16308121e4..c70e2ba1ffe 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.h +++ b/dbms/src/Storages/System/StorageSystemDatabases.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу databases, которая позволяет получить информацию о всех БД. +/** Implements `databases` system table, which allows you to get information about all databases. */ class StorageSystemDatabases : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemEvents.h b/dbms/src/Storages/System/StorageSystemEvents.h index 2308a9775cb..28b2b8f4600 100644 --- a/dbms/src/Storages/System/StorageSystemEvents.h +++ b/dbms/src/Storages/System/StorageSystemEvents.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу events, которая позволяет получить информацию для профайлинга. +/** Implements `events` system table, which allows you to obtain information for profiling. */ class StorageSystemEvents : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemFunctions.h b/dbms/src/Storages/System/StorageSystemFunctions.h index cf6150c2a51..ada8932dd8b 100644 --- a/dbms/src/Storages/System/StorageSystemFunctions.h +++ b/dbms/src/Storages/System/StorageSystemFunctions.h @@ -10,8 +10,8 @@ namespace DB class Context; -/** Реализует системную таблицу functions, которая позволяет получить список - * всех обычных и агрегатных функций. +/** Implements `functions`system table, which allows you to get a list + * all normal and aggregate functions. */ class StorageSystemFunctions : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemMetrics.h b/dbms/src/Storages/System/StorageSystemMetrics.h index c858f1d4e37..d3789622661 100644 --- a/dbms/src/Storages/System/StorageSystemMetrics.h +++ b/dbms/src/Storages/System/StorageSystemMetrics.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу metrics, которая позволяет получить информацию о работе сервера. +/** Implements `metrics` system table, which provides information about the operation of the server. */ class StorageSystemMetrics : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemNumbers.h b/dbms/src/Storages/System/StorageSystemNumbers.h index 293621e6642..18e0c609210 100644 --- a/dbms/src/Storages/System/StorageSystemNumbers.h +++ b/dbms/src/Storages/System/StorageSystemNumbers.h @@ -10,9 +10,9 @@ namespace DB class Context; -/** Реализует хранилище для системной таблицы Numbers. - * Таблица содержит единственный столбец number UInt64. - * Из этой таблицы можно прочитать все натуральные числа, начиная с 0 (до 2^64 - 1, а потом заново). +/** Implements a repository for the system table Numbers. + * The table contains the only column number UInt64. + * From this table, you can read all natural numbers, starting from 0 (to 2^64 - 1, and then again). */ class StorageSystemNumbers : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemOne.h b/dbms/src/Storages/System/StorageSystemOne.h index ec7085e9c48..b671a5bc8eb 100644 --- a/dbms/src/Storages/System/StorageSystemOne.h +++ b/dbms/src/Storages/System/StorageSystemOne.h @@ -10,10 +10,10 @@ namespace DB class Context; -/** Реализует хранилище для системной таблицы One. - * Таблица содержит единственный столбец dummy UInt8 и единственную строку со значением 0. - * Используется, если в запросе не указана таблица. - * Аналог таблицы DUAL в Oracle и MySQL. +/** Implements storage for the system table One. + * The table contains a single column of dummy UInt8 and a single row with a value of 0. + * Used when the table is not specified in the query. + * Analog of the DUAL table in Oracle and MySQL. */ class StorageSystemOne : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemProcesses.h b/dbms/src/Storages/System/StorageSystemProcesses.h index 11856c15911..fc0717efd0c 100644 --- a/dbms/src/Storages/System/StorageSystemProcesses.h +++ b/dbms/src/Storages/System/StorageSystemProcesses.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу processes, которая позволяет получить информацию о запросах, исполняющихся в данный момент. +/** Implements `processes` system table, which allows you to get information about the queries that are currently executing. */ class StorageSystemProcesses : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemReplicas.h b/dbms/src/Storages/System/StorageSystemReplicas.h index 15aad000cec..ed47d278320 100644 --- a/dbms/src/Storages/System/StorageSystemReplicas.h +++ b/dbms/src/Storages/System/StorageSystemReplicas.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу replicas, которая позволяет получить информацию о статусе реплицируемых таблиц. +/** Implements `replicas` system table, which provides information about the status of the replicated tables. */ class StorageSystemReplicas : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemReplicationQueue.h b/dbms/src/Storages/System/StorageSystemReplicationQueue.h index ebed5db28d3..2783d639628 100644 --- a/dbms/src/Storages/System/StorageSystemReplicationQueue.h +++ b/dbms/src/Storages/System/StorageSystemReplicationQueue.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу replication_queue, которая позволяет посмотреть очереди репликации для реплицируемых таблиц. +/** Implements the `replication_queue` system table, which allows you to view the replication queues for the replicated tables. */ class StorageSystemReplicationQueue : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemTables.h b/dbms/src/Storages/System/StorageSystemTables.h index 09678625655..3ffa715a225 100644 --- a/dbms/src/Storages/System/StorageSystemTables.h +++ b/dbms/src/Storages/System/StorageSystemTables.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу tables, которая позволяет получить информацию о всех таблицах. +/** Implements the system table `tables`, which allows you to get information about all tables. */ class StorageSystemTables : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.h b/dbms/src/Storages/System/StorageSystemZooKeeper.h index 4431d7b5ea8..e635cc3ea21 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.h +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.h @@ -10,7 +10,7 @@ namespace DB class Context; -/** Реализует системную таблицу zookeeper, которая позволяет просматривать данные в ZooKeeper в целях отладки. +/** Implements `zookeeper` system table, which allows you to view the data in ZooKeeper for debugging purposes. */ class StorageSystemZooKeeper : private ext::shared_ptr_helper, public IStorage { diff --git a/dbms/src/Storages/VirtualColumnFactory.h b/dbms/src/Storages/VirtualColumnFactory.h index ace05ca72c0..40ea9987a0c 100644 --- a/dbms/src/Storages/VirtualColumnFactory.h +++ b/dbms/src/Storages/VirtualColumnFactory.h @@ -5,8 +5,8 @@ namespace DB { -/** Знает имена и типы всех возможных виртуальных столбцов. - * Нужно для движков, перенаправляющих запрос в другие таблицы, не зная заранее, какие в них есть виртуальные столбцы. +/** Knows the names and types of all possible virtual columns. + * It is necessary for engines that redirect a request to other tables without knowing in advance what virtual columns they contain. */ class VirtualColumnFactory { diff --git a/dbms/src/TableFunctions/ITableFunction.h b/dbms/src/TableFunctions/ITableFunction.h index fb3653062be..d1d4e9be3c4 100644 --- a/dbms/src/TableFunctions/ITableFunction.h +++ b/dbms/src/TableFunctions/ITableFunction.h @@ -14,24 +14,24 @@ class IStorage; using StoragePtr = std::shared_ptr; -/** Интерфейс для табличных функций. +/** Interface for table functions. * - * Табличные функции не имеют отношения к другим функциям. - * Табличная функция может быть указана в секции FROM вместо [db.]table - * Табличная функция возвращает временный объект StoragePtr, который используется для выполнения запроса. + * Table functions are not relevant to other functions. + * The table function can be specified in the FROM section instead of the [db.]Table + * The table function returns a temporary StoragePtr object that is used to execute the query. * - * Пример: + * Example: * SELECT count() FROM remote('example01-01-1', merge, hits) - * - пойти на example01-01-1, в БД merge, таблицу hits. + * - go to `example01-01-1`, in `merge` database, `hits` table. */ class ITableFunction { public: - /// Получить основное имя функции. + /// Get the main function name. virtual std::string getName() const = 0; - /// Создать storage в соответствии с запросом + /// Create storage according to the query virtual StoragePtr execute(ASTPtr ast_function, Context & context) const = 0; virtual ~ITableFunction() {}; diff --git a/dbms/src/TableFunctions/TableFunctionFactory.h b/dbms/src/TableFunctions/TableFunctionFactory.h index 760b4ec8478..e0ab850145e 100644 --- a/dbms/src/TableFunctions/TableFunctionFactory.h +++ b/dbms/src/TableFunctions/TableFunctionFactory.h @@ -6,7 +6,7 @@ namespace DB { -/** Позволяет получить табличную функцию по ее имени. +/** Lets you get a table function by its name. */ class TableFunctionFactory { diff --git a/dbms/src/TableFunctions/TableFunctionMerge.h b/dbms/src/TableFunctions/TableFunctionMerge.h index cc83554de58..aa40a3e9bee 100644 --- a/dbms/src/TableFunctions/TableFunctionMerge.h +++ b/dbms/src/TableFunctions/TableFunctionMerge.h @@ -7,9 +7,9 @@ namespace DB { /* - * merge(db_name, tables_regexp)- создаёт временный StorageMerge. - * Cтруктура таблицы берётся из первой попавшейся таблицы, подходящей под регексп. - * Если такой таблицы нет - кидается исключение. + * merge (db_name, tables_regexp) - creates a temporary StorageMerge. + * The structure of the table is taken from the first table that came up, suitable for regexp. + * If there is no such table, an exception is thrown. */ class TableFunctionMerge: public ITableFunction { diff --git a/dbms/src/TableFunctions/TableFunctionRemote.h b/dbms/src/TableFunctions/TableFunctionRemote.h index 5dfa61b3ff8..56a1b21e20d 100644 --- a/dbms/src/TableFunctions/TableFunctionRemote.h +++ b/dbms/src/TableFunctions/TableFunctionRemote.h @@ -7,11 +7,11 @@ namespace DB { /* - * remote('address', db, table) - создаёт временный StorageDistributed. - * Чтобы получить структуру таблицы, делается запрос DESC TABLE на удалённый сервер. - * Например: - * SELECT count() FROM remote('example01-01-1', merge, hits) - пойти на example01-01-1, в БД merge, таблицу hits. - * В качестве имени хоста может быть указано также выражение, генерирующее множество шардов и реплик - см. ниже. + * remote ('address', db, table) - creates a temporary StorageDistributed. + * To get the table structure, a DESC TABLE request is made to the remote server. + * For example + * SELECT count() FROM remote('example01-01-1', merge, hits) - go to `example01-01-1`, in the merge database, the hits table. + * An expression that generates a lot of shards and replicas can also be specified as the host name - see below. */ class TableFunctionRemote : public ITableFunction { diff --git a/dbms/src/TableFunctions/TableFunctionShardByHash.h b/dbms/src/TableFunctions/TableFunctionShardByHash.h index 26233c8a744..64bc8531a5a 100644 --- a/dbms/src/TableFunctions/TableFunctionShardByHash.h +++ b/dbms/src/TableFunctions/TableFunctionShardByHash.h @@ -6,10 +6,10 @@ namespace DB { -/* shardByHash(cluster, 'key', db, table) - создаёт временный StorageDistributed, - * используя кластер cluster, и выбирая из него только один шард путём хэширования строки key. +/* shardByHash(cluster, 'key', db, table) - creates a temporary StorageDistributed, + * using the cluster `cluster`, and selecting from it only one shard by hashing the string key. * - * Аналогично функции remote, чтобы получить структуру таблицы, делается запрос DESC TABLE на удалённый сервер. + * Similarly to the `remote` function, to get the table structure, a DESC TABLE request is made to the remote server. */ class TableFunctionShardByHash : public ITableFunction { diff --git a/dbms/src/TableFunctions/getStructureOfRemoteTable.h b/dbms/src/TableFunctions/getStructureOfRemoteTable.h index 4ebdfca6afb..70b3d83b8f1 100644 --- a/dbms/src/TableFunctions/getStructureOfRemoteTable.h +++ b/dbms/src/TableFunctions/getStructureOfRemoteTable.h @@ -9,8 +9,8 @@ namespace DB class Cluster; class Context; -/// Узнать имена и типы столбцов таблицы на первом попавшемся сервере кластера. -/// Используется для реализации табличной функции remote и других. +/// Find the names and types of the table columns on any server in the cluster. +/// Used to implement the `remote` table function and others. NamesAndTypesList getStructureOfRemoteTable( const Cluster & cluster, const std::string & database,