Merge branch 'f1yegor-docs/translate'

This commit is contained in:
Alexey Milovidov 2017-03-26 04:28:54 +03:00
commit 6af570562b
107 changed files with 860 additions and 861 deletions

View File

@ -14,18 +14,18 @@
*/
#if USE_VECTORCLASS
#if __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshift-negative-value"
#endif
#if __clang__
#pragma clang diagnostic push
#pragma clang diagnostic ignored "-Wshift-negative-value"
#endif
#include <vectorf128.h>
#include <vectormath_exp.h>
#include <vectormath_trig.h>
#include <vectorf128.h>
#include <vectormath_exp.h>
#include <vectormath_trig.h>
#if __clang__
#pragma clang diagnostic pop
#endif
#if __clang__
#pragma clang diagnostic pop
#endif
#endif

View File

@ -10,18 +10,18 @@ namespace DB
{
/** Управление состояниями агрегатных функций делается нетривиальным образом:
* - память для них нужно выделяется в пуле,
* указатели на эти состояния могут передаваться между различными структурами данных,
* при этом нельзя сделать RAII-обёртки для каждого отдельного состояния.
* Подробнее см. Aggregator.h.
/** State management of aggregate functions is done in a non-trivial way:
* - the memory for them needs to be allocated in the pool,
* pointers to these states can be passed between different data structures,
* herewith, you can not make RAII wrappers for each individual state.
* For more information, see Aggregator.h.
*
* В связи с этим, возникают трудно-отлаживаемые баги.
* Для упрощения воспроизведения багов, была написана агрегатная функция debug,
* и её исходники решено не удалять после отладки.
* In this regard, there are difficult-debugging bugs.
* To simplify the playback of bugs, an aggregate `debug` function was written,
* and its source code is decided not to delete after debugging.
*
* Эта агрегатная функция принимает ноль аргументов и ничего не делает.
* Но у неё сделано состояние, которое нетривиально создаётся и уничтожается.
* This aggregate function takes zero arguments and does nothing.
* But it has a state that is non-trivially created and destroyed.
*/

View File

@ -22,7 +22,7 @@ namespace ErrorCodes
namespace
{
/// Ничего не проверяет.
/// Does not check anything.
std::string trimRight(const std::string & in, const char * suffix)
{
return in.substr(0, in.size() - strlen(suffix));
@ -161,14 +161,14 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(const String & name, cons
if ((recursion_level == 0) && endsWith(name, "State"))
{
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggState`, where `agg` is the name of another aggregate function.
AggregateFunctionPtr nested = get(trimRight(name, "State"), argument_types, recursion_level + 1);
return createAggregateFunctionState(nested);
}
if ((recursion_level <= 1) && endsWith(name, "Merge"))
{
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggMerge`, where `agg` is the name of another aggregate function.
if (argument_types.size() != 1)
throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const DataTypeAggregateFunction * function = typeid_cast<const DataTypeAggregateFunction *>(&*argument_types[0]);
@ -193,7 +193,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(const String & name, cons
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggIf`, where `agg` is the name of another aggregate function.
DataTypes nested_dt = argument_types;
nested_dt.pop_back();
AggregateFunctionPtr nested = get(trimRight(name, "If"), nested_dt, recursion_level + 1);
@ -202,7 +202,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(const String & name, cons
if ((recursion_level <= 3) && endsWith(name, "Array"))
{
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggArray`, where `agg` is the name of another aggregate function.
size_t num_agruments = argument_types.size();
DataTypes nested_arguments;
@ -214,7 +214,7 @@ AggregateFunctionPtr AggregateFunctionFactory::getImpl(const String & name, cons
throw Exception("Illegal type " + argument_types[i]->getName() + " of argument #" + toString(i + 1) +
" for aggregate function " + name + ". Must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
/// + 3, чтобы ни один другой модификатор не мог идти перед Array
/// + 3, so that no other modifier can go before the `Array`
AggregateFunctionPtr nested = get(trimRight(name, "Array"), nested_arguments, recursion_level + 3);
return createAggregateFunctionArray(nested);
}
@ -239,22 +239,22 @@ bool AggregateFunctionFactory::isAggregateFunctionName(const String & name, int
if (recursion_level == 0 && case_insensitive_aggregate_functions.count(Poco::toLower(name)))
return true;
/// Для агрегатных функций вида aggState, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggState`, where `agg` is the name of another aggregate function.
if ((recursion_level <= 0) && endsWith(name, "State"))
return isAggregateFunctionName(trimRight(name, "State"), recursion_level + 1);
/// Для агрегатных функций вида aggMerge, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggMerge`, where `agg` is the name of another aggregate function.
if ((recursion_level <= 1) && endsWith(name, "Merge"))
return isAggregateFunctionName(trimRight(name, "Merge"), recursion_level + 1);
/// Для агрегатных функций вида aggIf, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggIf`, where `agg` is the name of another aggregate function.
if ((recursion_level <= 2) && endsWith(name, "If"))
return isAggregateFunctionName(trimRight(name, "If"), recursion_level + 1);
/// Для агрегатных функций вида aggArray, где agg - имя другой агрегатной функции.
/// For aggregate functions of the form `aggArray`, where `agg` is the name of another aggregate function.
if ((recursion_level <= 3) && endsWith(name, "Array"))
{
/// + 3, чтобы ни один другой модификатор не мог идти перед Array
/// + 3, so that no other modifier can go before `Array`
return isAggregateFunctionName(trimRight(name, "Array"), recursion_level + 3);
}

View File

@ -31,7 +31,7 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
}
else if (argument_types.size() > 1)
{
/// Если аргументов несколько, то среди них недопустимо наличие кортежей.
/// If there are several arguments, then no tuples allowed among them.
for (const auto & type : argument_types)
if (typeid_cast<const DataTypeTuple *>(type.get()))
throw Exception("Tuple argument of function " + name + " must be the only argument",

View File

@ -8,8 +8,8 @@ namespace DB
namespace
{
/** DataForVariadic - структура с данными, которая будет использоваться для агрегатной функции uniq от множества аргументов.
* Отличается, например, тем, что использует тривиальную хэш-функцию, так как uniq от множества аргументов сначала самостоятельно их хэширует.
/** `DataForVariadic` is a data structure that will be used for `uniq` aggregate function of multiple arguments.
* It differs, for example, in that it uses a trivial hash function, since `uniq` of many arguments first hashes them out itself.
*/
template <typename Data, typename DataForVariadic>
@ -37,7 +37,7 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
}
else if (argument_types.size() > 1)
{
/// Если аргументов несколько, то среди них недопустимо наличие кортежей.
/// If there are several arguments, then no tuples allowed among them.
for (const auto & type : argument_types)
if (typeid_cast<const DataTypeTuple *>(type.get()))
throw Exception("Tuple argument of function " + name + " must be the only argument",
@ -75,7 +75,7 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
}
else if (argument_types.size() > 1)
{
/// Если аргументов несколько, то среди них недопустимо наличие кортежей.
/// If there are several arguments, then no tuples allowed among them.
for (const auto & type : argument_types)
if (typeid_cast<const DataTypeTuple *>(type.get()))
throw Exception("Tuple argument of function " + name + " must be the only argument",

View File

@ -260,7 +260,7 @@ private:
try
{
/// В этих потоках не будем принимать сигнал INT.
/// In these threads we do not accept INT signal.
sigset_t sig_set;
if (sigemptyset(&sig_set)
|| sigaddset(&sig_set, SIGINT)
@ -451,7 +451,7 @@ int mainEntryClickHouseBenchmark(int argc, char ** argv)
print_stacktrace = options.count("stacktrace");
/// Извлекаем settings and limits из полученных options
/// Extract `settings` and `limits` from received `options`
Settings settings;
#define EXTRACT_SETTING(TYPE, NAME, DEFAULT) \

View File

@ -39,11 +39,11 @@ static int sigtimedwait(const sigset_t *set, siginfo_t *info, const struct times
#endif
/** Пока существует объект этого класса - блокирует сигнал INT, при этом позволяет узнать, не пришёл ли он.
* Это нужно, чтобы можно было прервать выполнение запроса с помощью Ctrl+C.
* В один момент времени используйте только один экземпляр этого класса.
* Если метод check вернул true (пришёл сигнал), то следующие вызовы будут ждать следующий сигнал.
*/
/** As long as there exists an object of this class - it blocks the INT signal, at the same time it lets you know if it came.
  * This is necessary so that you can interrupt the execution of the request with Ctrl+C.
  * Use only one instance of this class at a time.
  * If `check` method returns true (the signal has arrived), the next call will wait for the next signal.
  */
class InterruptListener
{
private:
@ -94,7 +94,7 @@ public:
}
}
/// Можно прекратить блокировать сигнал раньше, чем в деструкторе.
/// You can stop blocking the signal earlier than in the destructor.
void unblock()
{
if (active)

View File

@ -113,7 +113,7 @@ void MultiplexedConnections::sendQuery(
{
if (settings == nullptr)
{
/// Каждый шард имеет один адрес.
/// Each shard has one address.
auto it = connections.begin();
for (size_t i = 0; i < shard_states.size(); ++i)
{
@ -127,7 +127,7 @@ void MultiplexedConnections::sendQuery(
}
else
{
/// Каждый шард имеет одну или несколько реплик.
/// Each shard has one or more replicas.
auto it = connections.begin();
for (const auto & shard_state : shard_states)
{
@ -244,7 +244,7 @@ Connection::Packet MultiplexedConnections::drain()
case Protocol::Server::Exception:
default:
/// Если мы получили исключение или неизвестный пакет, сохраняем его.
/// If we receive an exception or an unknown package, we save it.
res = std::move(packet);
break;
}
@ -281,8 +281,8 @@ void MultiplexedConnections::initFromShard(IConnectionPool * pool)
{
auto entries = pool->getMany(settings, pool_mode);
/// Если getMany() не выделил соединений и не кинул исключения, это значит, что была
/// установлена настройка skip_unavailable_shards. Тогда просто возвращаемся.
/// If getMany() did not allocate connections and did not throw exceptions, this means that
/// `skip_unavailable_shards` was set. Then just return.
if (entries.empty())
return;
@ -396,8 +396,8 @@ MultiplexedConnections::ReplicaMap::iterator MultiplexedConnections::waitForRead
Poco::Net::Socket::SocketList read_list;
read_list.reserve(active_connection_total_count);
/// Сначала проверяем, есть ли данные, которые уже лежат в буфере
/// хоть одного соединения.
/// First, we check if there are data already in the buffer
/// of at least one connection.
for (const auto & e : replica_map)
{
const ReplicaState & state = e.second;
@ -406,8 +406,8 @@ MultiplexedConnections::ReplicaMap::iterator MultiplexedConnections::waitForRead
read_list.push_back(connection->socket);
}
/// Если не было найдено никаких данных, то проверяем, есть ли соединения
/// готовые для чтения.
/// If no data was found, then we check if there are any connections
/// ready for reading.
if (read_list.empty())
{
Poco::Net::Socket::SocketList write_list;

View File

@ -1,3 +1,3 @@
<!-- Конфиг, устанавливающийся в /etc/clickhouse-client/. Он используется, если других конфигов не найдено. -->
<!-- Config set into /etc/clickhouse-client/. It's used if no other configs are found. -->
<config>
</config>

View File

@ -28,36 +28,35 @@ ColumnPtr ColumnAggregateFunction::convertToValues() const
const IAggregateFunction * function = func.get();
ColumnPtr res = function->getReturnType()->createColumn();
/** Если агрегатная функция возвращает нефинализированное состояние,
* то надо просто скопировать указатели на него а также разделяемое владение данными.
/** If the aggregate function returns an unfinalized/unfinished state,
* then you just need to copy pointers to it and also shared ownership of data.
*
* Также заменяем агрегатную функцию на вложенную.
* То есть, если этот столбец - состояния агрегатной функции aggState,
* то мы возвращаем такой же столбец, но с состояниями агрегатной функции agg.
* Это одни и те же состояния, меняем только функцию, которой они соответствуют.
* Also replace the aggregate function with the nested function.
* That is, if this column is the states of the aggregate function `aggState`,
* then we return the same column, but with the states of the aggregate function `agg`.
* These are the same states, changing only the function to which they correspond.
*
* Это довольно сложно для понимания.
* Пример, когда такое происходит:
* Further is quite difficult to understand.
* Example when this happens:
*
* SELECT k, finalizeAggregation(quantileTimingState(0.5)(x)) FROM ... GROUP BY k WITH TOTALS
*
* Здесь вычисляется агрегатная функция quantileTimingState.
* Её тип возвращаемого значения:
* AggregateFunction(quantileTiming(0.5), UInt64).
* Из-за наличия WITH TOTALS, при агрегации будут сохранены состояния этой агрегатной функции
* в столбце ColumnAggregateFunction, имеющего тип
* This calculates the aggregate function `quantileTimingState`.
* Its return type AggregateFunction(quantileTiming(0.5), UInt64)`.
* Due to the presence of WITH TOTALS, during aggregation the states of this aggregate function will be stored
* in the ColumnAggregateFunction column of type
* AggregateFunction(quantileTimingState(0.5), UInt64).
* Затем в TotalsHavingBlockInputStream у него будет вызван метод convertToValues,
* чтобы получить "готовые" значения.
* Но он всего лишь преобразует столбец типа
* AggregateFunction(quantileTimingState(0.5), UInt64)
* в AggregateFunction(quantileTiming(0.5), UInt64)
* - в такие же состояния.
* Then, in `TotalsHavingBlockInputStream`, it will be called `convertToValues` method,
* to get the "ready" values.
* But it just converts a column of type
* `AggregateFunction(quantileTimingState(0.5), UInt64)`
* into `AggregateFunction(quantileTiming(0.5), UInt64)`
* - in the same states.
*
* Затем будет вычислена функция finalizeAggregation, которая позовёт convertToValues уже от результата.
* И это преобразует столбец типа
* Then `finalizeAggregation` function will be calculated, which will call `convertToValues` already on the result.
* And this converts a column of type
* AggregateFunction(quantileTiming(0.5), UInt64)
* в UInt16 - уже готовый результат работы quantileTiming.
* into UInt16 - already finished result of `quantileTiming`.
*/
if (const AggregateFunctionState * function_state = typeid_cast<const AggregateFunctionState *>(function))
{
@ -132,7 +131,7 @@ ColumnPtr ColumnAggregateFunction::filter(const Filter & filter, ssize_t result_
if (filter[i])
res_data.push_back(getData()[i]);
/// Для экономии оперативки в случае слишком сильной фильтрации.
/// To save RAM in case of too strong filtering.
if (res_data.size() * 2 < res_data.capacity())
res_data = Container_t(res_data.cbegin(), res_data.cend());

View File

@ -423,7 +423,7 @@ ColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_size_hin
Offsets_t & res_string_offsets = typeid_cast<ColumnString &>(res->getData()).getOffsets();
Offsets_t & res_offsets = res->getOffsets();
if (result_size_hint < 0) /// Остальные случаи не рассматриваем.
if (result_size_hint < 0) /// Other cases are not considered.
{
res_chars.reserve(src_chars.size());
res_string_offsets.reserve(src_string_offsets.size());
@ -438,12 +438,12 @@ ColumnPtr ColumnArray::filterString(const Filter & filt, ssize_t result_size_hin
for (size_t i = 0; i < col_size; ++i)
{
/// Количество строк в массиве.
/// Number of rows in the array.
size_t array_size = src_offsets[i] - prev_src_offset;
if (filt[i])
{
/// Если массив не пуст - копируем внутренности.
/// If the array is not empty - copy content.
if (array_size)
{
size_t chars_to_copy = src_string_offsets[array_size + prev_src_offset - 1] - prev_src_string_offset;
@ -494,8 +494,8 @@ ColumnPtr ColumnArray::filterGeneric(const Filter & filt, ssize_t result_size_hi
ssize_t nested_result_size_hint = 0;
if (result_size_hint < 0)
nested_result_size_hint = result_size_hint;
else if (result_size_hint && result_size_hint < 1000000000 && data->size() < 1000000000) /// Избегаем переполнения.
nested_result_size_hint = result_size_hint * data->size() / size;
else if (result_size_hint && result_size_hint < 1000000000 && data->size() < 1000000000) /// Avoid overflow.
nested_result_size_hint = result_size_hint * data->size() / size;
res->data = data->filter(nested_filt, nested_result_size_hint);
@ -584,7 +584,7 @@ void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_h
ColumnPtr ColumnArray::replicate(const Offsets_t & replicate_offsets) const
{
/// Не получается реализовать в общем случае.
/// It does not work out in general case.
if (typeid_cast<const ColumnUInt8 *>(data.get())) return replicateNumber<UInt8>(replicate_offsets);
if (typeid_cast<const ColumnUInt16 *>(data.get())) return replicateNumber<UInt16>(replicate_offsets);
@ -688,11 +688,11 @@ ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) cons
for (size_t i = 0; i < col_size; ++i)
{
/// Насколько размножить массив.
/// How much to replicate the array.
size_t size_to_replicate = replicate_offsets[i] - prev_replicate_offset;
/// Количество строк в массиве.
/// The number of rows in the array.
size_t value_size = src_offsets[i] - prev_src_offset;
/// Количество символов в строках массива, включая нулевые байты.
/// Number of characters in rows of the array, including zero/null bytes.
size_t sum_chars_size = value_size == 0 ? 0 : (src_string_offsets[prev_src_offset + value_size - 1] - prev_src_string_offset);
for (size_t j = 0; j < size_to_replicate; ++j)
@ -703,7 +703,7 @@ ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) cons
size_t prev_src_string_offset_local = prev_src_string_offset;
for (size_t k = 0; k < value_size; ++k)
{
/// Размер одной строки.
/// Size of one row.
size_t chars_size = src_string_offsets[k + prev_src_offset] - prev_src_string_offset_local;
current_res_string_offset += chars_size;
@ -712,7 +712,7 @@ ColumnPtr ColumnArray::replicateString(const Offsets_t & replicate_offsets) cons
prev_src_string_offset_local += chars_size;
}
/// Копирование символов массива строк.
/// Copies the characters of the array of rows.
res_chars.resize(res_chars.size() + sum_chars_size);
memcpySmallAllowReadWriteOverflow15(
&res_chars[res_chars.size() - sum_chars_size], &src_chars[prev_src_string_offset], sum_chars_size);

View File

@ -12,9 +12,9 @@ size_t countBytesInFilter(const IColumn::Filter & filt)
{
size_t count = 0;
/** NOTE: По идее, filt должен содержать только нолики и единички.
* Но, на всякий случай, здесь используется условие > 0 (на знаковые байты).
* Лучше было бы использовать != 0, то это не позволяет SSE2.
/** NOTE: In theory, `filt` should only contain zeros and ones.
* But, just in case, here the condition > 0 (to signed bytes) is used.
* It would be better to use != 0, then this does not allow SSE2.
*/
const Int8 * pos = reinterpret_cast<const Int8 *>(&filt[0]);
@ -69,7 +69,7 @@ void filterArraysImpl(
if (result_size_hint < 0)
res_elems.reserve(src_elems.size());
else if (result_size_hint < 1000000000 && src_elems.size() < 1000000000) /// Избегаем переполнения.
else if (result_size_hint < 1000000000 && src_elems.size() < 1000000000) /// Avoid overflow.
res_elems.reserve((result_size_hint * src_elems.size() + size - 1) / size);
}
@ -166,7 +166,7 @@ void filterArraysImpl(
}
/// Явные инстанцирования - чтобы не размещать реализацию функции выше в заголовочном файле.
/// Explicit instantiations - not to place the implementation of the function above in the header file.
template void filterArraysImpl<UInt8>(
const PaddedPODArray<UInt8> &, const IColumn::Offsets_t &,
PaddedPODArray<UInt8> &, IColumn::Offsets_t &,

View File

@ -44,8 +44,8 @@ void FileChecker::update(const Files::const_iterator & begin, const Files::const
bool FileChecker::check() const
{
/** Читаем файлы заново при каждом вызове check - чтобы не нарушать константность.
* Метод check вызывается редко.
/** Read the files again every time you call `check` - so as not to violate the constancy.
* `check` method is rarely called.
*/
Map local_map;
load(local_map);
@ -92,7 +92,7 @@ void FileChecker::save() const
{
WriteBufferFromFile out(tmp_files_info_path);
/// Столь сложная структура JSON-а - для совместимости со старым форматом.
/// So complex JSON structure - for compatibility with the old format.
writeCString("{\"yandex\":{", out);
for (auto it = map.begin(); it != map.end(); ++it)
@ -100,7 +100,7 @@ void FileChecker::save() const
if (it != map.begin())
writeString(",", out);
/// escapeForFileName на самом деле не нужен. Но он оставлен для совместимости со старым кодом.
/// `escapeForFileName` is not really needed. But it is left for compatibility with the old code.
writeJSONString(escapeForFileName(it->first), out);
writeString(":{\"size\":\"", out);
writeIntText(it->second, out);
@ -136,7 +136,7 @@ void FileChecker::load(Map & map) const
ReadBufferFromFile in(files_info_path);
WriteBufferFromString out(content);
/// Библиотека JSON не поддерживает пробельные символы. Удаляем их. Неэффективно.
/// The JSON library does not support whitespace. We delete them. Ineffective.
while (!in.eof())
{
char c;

View File

@ -62,10 +62,10 @@ namespace
}
};
/// По этим кодам возврата из дочернего процесса мы узнаем (наверняка) об ошибках при его создании.
/// By these return codes from the child process, we learn (for sure) about errors when creating it.
enum class ReturnCodes : int
{
CANNOT_DUP_STDIN = 42, /// Значение не принципиально, но выбрано так, чтобы редко конфликтовать с кодом возврата программы.
CANNOT_DUP_STDIN = 42, /// The value is not important, but it is chosen so that it's rare to conflict with the program return code.
CANNOT_DUP_STDOUT = 43,
CANNOT_DUP_STDERR = 44,
CANNOT_EXEC = 45,
@ -79,10 +79,10 @@ namespace DB
std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, char * const argv[], bool pipe_stdin_only)
{
/** Тут написано, что при обычном вызове vfork, есть шанс deadlock-а в многопоточных программах,
* из-за резолвинга символов в shared-библиотеке:
/** Here it is written that with a normal call `vfork`, there is a chance of deadlock in multithreaded programs,
* because of the resolving of characters in the shared library
* http://www.oracle.com/technetwork/server-storage/solaris10/subprocess-136439.html
* Поэтому, отделим резолвинг символа от вызова.
* Therefore, separate the resolving of the symbol from the call.
*/
static void * real_vfork = dlsym(RTLD_DEFAULT, "vfork");
@ -100,12 +100,12 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
if (0 == pid)
{
/// Находимся в свежесозданном процессе.
/// We are in the freshly created process.
/// Почему _exit а не exit? Потому что exit вызывает atexit и деструкторы thread local storage.
/// А там куча мусора (в том числе, например, блокируется mutex). А это нельзя делать после vfork - происходит deadlock.
/// Why `_exit` and not `exit`? Because `exit` calls `atexit` and destructors of thread local storage.
/// And there is a lot of garbage (including, for example, mutex is blocked). And this can not be done after `vfork` - deadlock happens.
/// Заменяем файловые дескрипторы на концы наших пайпов.
/// Replace the file descriptors with the ends of our pipes.
if (STDIN_FILENO != dup2(pipe_stdin.read_fd, STDIN_FILENO))
_exit(int(ReturnCodes::CANNOT_DUP_STDIN));
@ -119,14 +119,14 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
}
execv(filename, argv);
/// Если процесс запущен, то execv не возвращает сюда.
/// If the process is running, then `execv` does not return here.
_exit(int(ReturnCodes::CANNOT_EXEC));
}
std::unique_ptr<ShellCommand> res(new ShellCommand(pid, pipe_stdin.write_fd, pipe_stdout.read_fd, pipe_stderr.read_fd));
/// Теперь владение файловыми дескрипторами передано в результат.
/// Now the ownership of the file descriptors is passed to the result.
pipe_stdin.write_fd = -1;
pipe_stdout.read_fd = -1;
pipe_stderr.read_fd = -1;
@ -137,8 +137,8 @@ std::unique_ptr<ShellCommand> ShellCommand::executeImpl(const char * filename, c
std::unique_ptr<ShellCommand> ShellCommand::execute(const std::string & command, bool pipe_stdin_only)
{
/// Аргументы в неконстантных кусках памяти (как требуется для execv).
/// Причём, их копирование должно быть совершено раньше вызова vfork, чтобы после vfork делать минимум вещей.
/// Arguments in non-constant chunks of memory (as required for `execv`).
/// Moreover, their copying must be done before calling `vfork`, so after `vfork` do a minimum of things.
std::vector<char> argv0("sh", "sh" + strlen("sh") + 1);
std::vector<char> argv1("-c", "-c" + strlen("-c") + 1);
std::vector<char> argv2(command.data(), command.data() + command.size() + 1);

View File

@ -27,7 +27,7 @@ std::string StackTrace::toString() const
{
for (size_t i = 0, size = frames_size; i < size; ++i)
{
/// Делаем demangling имён. Имя находится в скобках, до символа '+'.
/// We do "demangling" of names. The name is in parenthesis, before the '+' character.
char * name_start = nullptr;
char * name_end = nullptr;

View File

@ -93,7 +93,7 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va
asts.insert(asts.begin(), column_value);
}
/// Проверка, что функция зависит только от заданных столбцов
/// Verifying that the function depends only on the specified columns
static bool isValidFunction(ASTPtr expression, const NameSet & columns)
{
for (size_t i = 0; i < expression->children.size(); ++i)
@ -108,7 +108,7 @@ static bool isValidFunction(ASTPtr expression, const NameSet & columns)
return true;
}
/// Извлечь все подфункции главной конъюнкции, но зависящие только от заданных столбцов
/// Extract all subfunctions of the main conjunction, but depending only on the specified columns
static void extractFunctions(ASTPtr expression, const NameSet & columns, std::vector<ASTPtr> & result)
{
const ASTFunction * function = typeid_cast<const ASTFunction *>(&*expression);
@ -123,7 +123,7 @@ static void extractFunctions(ASTPtr expression, const NameSet & columns, std::ve
}
}
/// Построить конъюнкцию из заданных функций
/// Construct a conjunction from given functions
static ASTPtr buildWhereExpression(const ASTs & functions)
{
if (functions.size() == 0) return nullptr;
@ -148,7 +148,7 @@ bool filterBlockWithQuery(ASTPtr query, Block & block, const Context & context)
for (const auto & it : block.getColumnsList())
columns.insert(it.name);
/// Составим выражение, вычисляющее выражения в WHERE и PREWHERE, зависящие только от имеющихся столбцов.
/// We will create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns.
std::vector<ASTPtr> functions;
if (select.where_expression)
extractFunctions(select.where_expression, columns, functions);
@ -158,12 +158,12 @@ bool filterBlockWithQuery(ASTPtr query, Block & block, const Context & context)
if (!expression_ast)
return false;
/// Распарсим и вычислим выражение.
/// Let's parse and calculate the expression.
ExpressionAnalyzer analyzer(expression_ast, context, {}, block.getColumnsList());
ExpressionActionsPtr actions = analyzer.getActions(false);
actions->execute(block);
/// Отфильтруем блок.
/// Filter the block.
String filter_column_name = expression_ast->getColumnName();
ColumnPtr filter_column = block.getByName(filter_column_name).column;
if (auto converted = filter_column->convertToFullColumnIfConst())

View File

@ -44,7 +44,7 @@ std::string unescapeForFileName(const std::string & s)
res += *pos;
else
{
/// пропустим '%'
/// skip '%'
if (++pos == end) break;
char val = unhex(*pos) * 16;

View File

@ -19,9 +19,9 @@ bool isLocalAddress(const Poco::Net::SocketAddress & address)
return interfaces.end() != std::find_if(interfaces.begin(), interfaces.end(),
[&] (const Poco::Net::NetworkInterface & interface)
{
/** Сравниваем адреса без учёта scope.
* Теоретически, это может быть неверно - зависит от настройки route
* - через какой интерфейс мы на самом деле будем обращаться к заданному адресу.
/** Compare the addresses without taking into account `scope`.
* Theoretically, this may not be correct - depends on `route` setting
* - through which interface we will actually access the specified address.
*/
return interface.address().length() == address.host().length()
&& 0 == memcmp(interface.address().addr(), address.host().addr(), address.host().length());

View File

@ -41,8 +41,8 @@ static void localBackupImpl(Poco::Path source_path, Poco::Path destination_path,
std::string source_str = source.toString();
std::string destination_str = destination.toString();
/** Пытаемся создать hard link.
* Если он уже существует, то проверим, что source и destination указывают на один и тот же inode.
/** We are trying to create a hard link.
* If it already exists, we check that source and destination point to the same inode.
*/
if (0 != link(source_str.c_str(), destination_str.c_str()))
{
@ -84,9 +84,9 @@ void localBackup(Poco::Path source_path, Poco::Path destination_path)
size_t try_no = 0;
const size_t max_tries = 10;
/** В директории могут постоянно добавляться и удаляться файлы.
* Если какой-то файл удалился во время попытки сделать бэкап, то повторим попытку снова,
* так как важно учесть какие-нибудь новые файлы, который могли появиться.
/** Files in the directory can be permanently added and deleted.
* If some file is deleted during an attempt to make a backup, then try again,
* because it's important to take into account any new files that might appear.
*/
while (true)
{

View File

@ -1,4 +1,4 @@
/// Взято из SMHasher.
/// Taken from SMHasher.
#include "AvalancheTest.h"

View File

@ -1,4 +1,4 @@
/// Взято из SMHasher.
/// Taken from SMHasher.
//-----------------------------------------------------------------------------
// Flipping a single bit of a key should cause an "avalanche" of changes in

View File

@ -1,4 +1,4 @@
/// Взято из SMHasher.
/// Taken from SMHasher.
#ifdef __APPLE__
#include <common/apple_rt.h>
#endif

View File

@ -1,4 +1,4 @@
/// Взято из SMHasher.
/// Taken from SMHasher.
#pragma once

View File

@ -1,4 +1,4 @@
/** Запускать так (пример):
/** Run this (example)
* ./arena_with_free_lists 5000000 < ../../Server/data/test/hits/20140317_20140323_2_178_4/Title.bin
*/
@ -25,7 +25,7 @@
using namespace DB;
/// Реализация ArenaWithFreeLists, которая содержит баг. Используется, чтобы воспроизвести баг.
/// Implementation of ArenaWithFreeLists, which contains a bug. Used to reproduce the bug.
#if USE_BAD_ARENA
class ArenaWithFreeLists : private Allocator<false>
@ -47,7 +47,7 @@ private:
static auto sizeToPreviousPowerOfTwo(const int size)
{
return _bit_scan_reverse(size - 1);
/// Баг расположен в строчке выше. Если заменить на следующую строчку, то баг исправляется.
/// The bug is located in the line above. If you change to the next line, then the bug is fixed.
//return size <= 1 ? 0 : _bit_scan_reverse(size - 1);
}
@ -110,7 +110,7 @@ public:
block->next = old;
}
/// Размер выделенного пула в байтах
/// Size of the allocated pool in bytes
size_t size() const
{
return pool.size();
@ -120,7 +120,7 @@ public:
#endif
/// Небольшой кусочек, выдранный из CacheDictionary. Используется только для того, чтобы продемонстрировать проблему.
/// A small piece copied from the CacheDictionary. It is used only to demonstrate the problem.
struct Dictionary
{
template <typename Value> using ContainerType = Value[];
@ -158,7 +158,7 @@ struct Dictionary
std::unique_ptr<ArenaWithFreeLists> string_arena;
/// Эта функция компилируется в точно такой же машинный код, как в продакшене, когда был баг.
/// This function is compiled into exactly the same machine code as in production, when there was a bug.
void NO_INLINE setAttributeValue(Attribute & attribute, const UInt64 idx, const Field & value) const
{
switch (attribute.type)

View File

@ -1,4 +1,4 @@
/// Баг в GCC: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=59124
/// Bug in GCC: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=59124
#if !__clang__
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Warray-bounds"

View File

@ -9,7 +9,7 @@
#include <DB/Common/HashTable/Hash.h>
#include <DB/Common/Stopwatch.h>
#include "AvalancheTest.h" /// Взято из SMHasher.
#include "AvalancheTest.h" /// Taken from SMHasher.
#ifdef __APPLE__
@ -214,7 +214,7 @@ void report(const char * name, size_t n, double elapsed, UInt64 tsc_diff, size_t
template <size_t Func(UInt64)>
static inline void test(size_t n, const UInt64 * data, const char * name)
{
/// throughput. Вычисления хэш-функций от разных значений могут перекрываться.
/// throughput. Calculations of hash functions from different values may overlap.
{
Stopwatch watch;
@ -232,7 +232,7 @@ static inline void test(size_t n, const UInt64 * data, const char * name)
report(name, n, watch.elapsedSeconds(), tsc_diff, res);
}
/// latency. Чтобы вычислить следующее значение, надо сначала вычислить предыдущее. Добавляется latency L1-кэша.
/// latency. To calculate the next value, you must first calculate the previous one. The latency of the L1 cache is added.
{
Stopwatch watch;
@ -254,7 +254,7 @@ static inline void test(size_t n, const UInt64 * data, const char * name)
report(name, n, watch.elapsedSeconds(), tsc_diff, res);
}
/// quality. Методы взяты из SMHasher.
/// quality. Methods are taken from SMHasher.
{
auto wrapper = [](const void * blob, const int len, const uint32_t seed, void * out)
{

View File

@ -140,7 +140,7 @@ void aggregate3(Map & local_map, Map & global_map, Mutex & mutex, Source::const_
if (found != local_map.end())
++found->second;
else if (local_map.size() < threshold)
++local_map[*it]; /// TODO Можно было бы делать один lookup, а не два.
++local_map[*it]; /// TODO You could do one lookup, not two.
else
{
if (mutex.try_lock())
@ -228,7 +228,7 @@ void aggregate5(Map & local_map, MapSmallLocks & global_map, Source::const_itera
if (found != local_map.end())
++found->second;
else if (local_map.size() < threshold)
++local_map[*it]; /// TODO Можно было бы делать один lookup, а не два.
++local_map[*it]; /// TODO You could do one lookup, not two.
else
{
SmallScopedLock lock;
@ -274,9 +274,9 @@ int main(int argc, char ** argv)
if (!method || method == 1)
{
/** Вариант 1.
* В разных потоках агрегируем независимо в разные хэш-таблицы.
* Затем сливаем их вместе.
/** Option 1.
* In different threads, we aggregate independently into different hash tables.
* Then merge them together.
*/
std::vector<Map> maps(num_threads);
@ -330,7 +330,7 @@ int main(int argc, char ** argv)
if (!method || method == 12)
{
/** То же самое, но с оптимизацией для подряд идущих одинаковых значений.
/** The same, but with optimization for consecutive identical values.
*/
std::vector<Map> maps(num_threads);
@ -385,11 +385,11 @@ int main(int argc, char ** argv)
if (!method || method == 11)
{
/** Вариант 11.
* То же, что вариант 1, но при мердже, изменён порядок циклов,
* что потенциально может дать лучшую кэш-локальность.
/** Option 11.
* Same as option 1, but with merge, the order of the cycles is changed,
* which potentially can give better cache locality.
*
* На практике, разницы нет.
* In practice, there is no difference.
*/
std::vector<Map> maps(num_threads);
@ -460,12 +460,12 @@ int main(int argc, char ** argv)
if (!method || method == 2)
{
/** Вариант 2.
* В разных потоках агрегируем независимо в разные two-level хэш-таблицы.
* Затем сливаем их вместе, распараллелив по bucket-ам первого уровня.
* При использовании хэш-таблиц больших размеров (10 млн. элементов и больше),
* и большого количества потоков (8-32), слияние является узким местом,
* и преимущество в производительности достигает 4 раз.
/** Option 2.
* In different threads, we aggregate independently into different two-level hash tables.
* Then merge them together, parallelizing by the first level buckets.
* When using hash tables of large sizes (10 million elements or more),
* and a large number of threads (8-32), the merge is a bottleneck,
* and has a performance advantage of 4 times.
*/
std::vector<MapTwoLevel> maps(num_threads);
@ -576,14 +576,14 @@ int main(int argc, char ** argv)
if (!method || method == 3)
{
/** Вариант 3.
* В разных потоках агрегируем независимо в разные хэш-таблицы,
* пока их размер не станет достаточно большим.
* Если размер локальной хэш-таблицы большой, и в ней нет элемента,
* то вставляем его в одну глобальную хэш-таблицу, защищённую mutex-ом,
* а если mutex не удалось захватить, то вставляем в локальную.
* Затем сливаем все локальные хэш-таблицы в глобальную.
* Этот метод плохой - много contention-а.
/** Option 3.
* In different threads, we aggregate independently into different hash tables,
* until their size becomes large enough.
* If the size of the local hash table is large, and there is no element in it,
* then we insert it into one global hash table, protected by mutex,
* and if mutex failed to capture, then insert it into the local one.
* Then merge all the local hash tables to the global one.
* This method is bad - a lot of contention.
*/
std::vector<Map> local_maps(num_threads);
@ -646,10 +646,10 @@ int main(int argc, char ** argv)
if (!method || method == 33)
{
/** Вариант 33.
* В разных потоках агрегируем независимо в разные хэш-таблицы,
* пока их размер не станет достаточно большим.
* Затем сбрасываем данные в глобальную хэш-таблицу, защищённую mutex-ом, и продолжаем.
/** Option 33.
* In different threads, we aggregate independently into different hash tables,
* until their size becomes large enough.
* Then we insert the data to the global hash table, protected by mutex, and continue.
*/
std::vector<Map> local_maps(num_threads);
@ -712,13 +712,13 @@ int main(int argc, char ** argv)
if (!method || method == 4)
{
/** Вариант 4.
* В разных потоках агрегируем независимо в разные хэш-таблицы,
* пока их размер не станет достаточно большим.
* Если размер локальной хэш-таблицы большой, и в ней нет элемента,
* то вставляем его в одну из 256 глобальных хэш-таблиц, каждая из которых под своим mutex-ом.
* Затем сливаем все локальные хэш-таблицы в глобальную.
* Этот метод не такой уж плохой при большом количестве потоков, но хуже второго.
/** Option 4.
* In different threads, we aggregate independently into different hash tables,
* until their size becomes large enough.
* If the size of the local hash table is large, and there is no element in it,
* then insert it into one of 256 global hash tables, each of which is under its mutex.
* Then merge all local hash tables into the global one.
* This method is not so bad with a lot of threads, but worse than the second one.
*/
std::vector<Map> local_maps(num_threads);
@ -783,13 +783,13 @@ int main(int argc, char ** argv)
/* if (!method || method == 5)
{
*/ /** Вариант 5.
* В разных потоках агрегируем независимо в разные хэш-таблицы,
* пока их размер не станет достаточно большим.
* Если размер локальной хэш-таблицы большой, и в ней нет элемента,
* то вставляем его в одну глобальную хэш-таблицу, содержащую маленькие защёлки в каждой ячейке,
* а если защёлку не удалось захватить, то вставляем в локальную.
* Затем сливаем все локальные хэш-таблицы в глобальную.
*/ /** Option 5.
* In different threads, we aggregate independently into different hash tables,
* until their size becomes large enough.
* If the size of the local hash table is large and there is no element in it,
* then insert it into one global hash table containing small latches in each cell,
* and if the latch can not be captured, then insert it into the local one.
* Then merge all local hash tables into the global one.
*/
/*
Map local_maps[num_threads];
@ -850,10 +850,10 @@ int main(int argc, char ** argv)
/*if (!method || method == 6)
{
*//** Вариант 6.
* В разных потоках агрегируем независимо в разные хэш-таблицы.
* Затем "сливаем" их, проходя по ним в одинаковом порядке ключей.
* Довольно тормозной вариант.
*//** Option 6.
* In different threads, we aggregate independently into different hash tables.
* Then "merge" them, passing them in the same order of the keys.
* Quite a slow option.
*/
/*
std::vector<Map> maps(num_threads);

View File

@ -139,7 +139,7 @@ struct MergeSequential
};
template <typename Map>
struct MergeSequentialTransposed /// На практике не лучше обычного.
struct MergeSequentialTransposed /// In practice not better than usual.
{
template <typename Merger>
static void NO_INLINE execute(Map ** source_maps, size_t num_maps, Map *& result_map,

View File

@ -6,7 +6,7 @@
#include <DB/Common/SipHash.h>
/// Адаптированный вариант https://www.131002.net/siphash/siphash24.c
/// Adapted version https://www.131002.net/siphash/siphash24.c
/*
SipHash-2-4 output with

View File

@ -8,7 +8,7 @@
#include <DB/Common/Stopwatch.h>
/** Тестировать так:
/** Test this way:
*
* clickhouse-client --query="SELECT SearchPhrase AS k FROM test.hits WHERE k != ''" > phrases.tsv
* clickhouse-client --query="SELECT URL AS k FROM test.hits" > urls.tsv

View File

@ -28,8 +28,8 @@ namespace ErrorCodes
void Block::addDefaults(const NamesAndTypesList & required_columns)
{
/// Для недостающих столбцов из вложенной структуры нужно создавать не столбец пустых массивов, а столбец массивов правильных длин.
/// Сначала запомним столбцы смещений для всех массивов в блоке.
/// For missing columns of nested structure, you need to create not a column of empty arrays, but a column of arrays of correct lengths.
/// First, remember the offset columns for all arrays in the block.
std::map<String, ColumnPtr> offset_columns;
for (const auto & elem : data)
@ -39,7 +39,7 @@ void Block::addDefaults(const NamesAndTypesList & required_columns)
String offsets_name = DataTypeNested::extractNestedTableName(elem.name);
auto & offsets_column = offset_columns[offsets_name];
/// Если почему-то есть разные столбцы смещений для одной вложенной структуры, то берём непустой.
/// If for some reason there are different displacement columns for one nested structure, then we take nonempty.
if (!offsets_column || offsets_column->empty())
offsets_column = array->getOffsetsColumn();
}
@ -70,8 +70,8 @@ void Block::addDefaults(const NamesAndTypesList & required_columns)
}
else
{
/** Нужно превратить константный столбец в полноценный, так как в части блоков (из других кусков),
* он может быть полноценным (а то интерпретатор может посчитать, что он константный везде).
/** It is necessary to turn a constant column into a full column, since in part of blocks (from other parts),
* it can be full (or the interpreter may decide that it is constant everywhere).
*/
column_to_add.column = dynamic_cast<IColumnConst &>(
*column_to_add.type->createConstColumn(
@ -389,7 +389,7 @@ NamesAndTypesList Block::getColumnsList() const
void Block::checkNestedArraysOffsets() const
{
/// Указатели на столбцы-массивы, для проверки равенства столбцов смещений во вложенных структурах данных
/// Pointers to array columns, to check the equality of offset columns in nested data structures
using ArrayColumns = std::map<String, const ColumnArray *>;
ArrayColumns array_columns;
@ -423,7 +423,7 @@ void Block::checkNestedArraysOffsets() const
void Block::optimizeNestedArraysOffsets()
{
/// Указатели на столбцы-массивы, для проверки равенства столбцов смещений во вложенных структурах данных
/// Pointers to array columns, to check the equality of offset columns in nested data structures
using ArrayColumns = std::map<String, ColumnArray *>;
ArrayColumns array_columns;
@ -450,7 +450,7 @@ void Block::optimizeNestedArraysOffsets()
if (!it->second->hasEqualOffsets(*column_array))
throw Exception("Sizes of nested arrays do not match", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH);
/// делаем так, чтобы столбцы смещений массивов внутри одной вложенной таблицы указывали в одно место
/// make columns of arrays offsets inside one nested table point to the same place
column_array->getOffsetsColumn() = it->second->getOffsetsColumn();
}
}
@ -479,8 +479,8 @@ bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs)
void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out_lhs_diff, std::string & out_rhs_diff)
{
/// Традиционная задача: наибольшая общая подпоследовательность (LCS).
/// Полагаем, что порядок важен. Если это когда-то станет не так, упростим: например, сделаем 2 set'а.
/// The traditional task: the largest common subsequence (LCS).
/// Assume that order is important. If this becomes wrong once, let's simplify it: for example, make 2 sets.
std::vector<std::vector<int>> lcs(lhs.columns() + 1);
for (auto & v : lcs)
@ -497,7 +497,7 @@ void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out
}
}
/// Теперь идем обратно и собираем ответ.
/// Now go back and collect the answer.
ColumnsWithTypeAndName left_columns;
ColumnsWithTypeAndName right_columns;
size_t l = lhs.columns();
@ -506,15 +506,15 @@ void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out
{
if (lhs.safeGetByPosition(l - 1) == rhs.safeGetByPosition(r - 1))
{
/// Данный элемент в обеих последовательностях, значит, в diff не попадает.
/// This element is in both sequences, so it does not get into `diff`.
--l;
--r;
}
else
{
/// Маленькая эвристика: чаще всего используется при получении разницы для (expected_block, actual_block).
/// Поэтому предпочтение будем отдавать полю, которое есть в левом блоке (expected_block), поэтому
/// в diff попадет столбец из actual_block.
/// Small heuristics: most often used when getting a difference for (expected_block, actual_block).
/// Therefore, the preference will be given to the field, which is in the left block (expected_block), therefore
/// in `diff` the column from `actual_block` will get.
if (lcs[l][r - 1] >= lcs[l - 1][r])
right_columns.push_back(rhs.safeGetByPosition(--r));
else

View File

@ -17,10 +17,10 @@ namespace ErrorCodes
}
/// Записать значения в бинарном виде. NOTE: Можно было бы использовать protobuf, но он был бы overkill для данного случая.
/// Write values in binary form. NOTE: You could use protobuf, but it would be overkill for this case.
void BlockInfo::write(WriteBuffer & out) const
{
/// Набор пар FIELD_NUM, значение в бинарном виде. Затем 0.
/// Set of pairs `FIELD_NUM`, value in binary form. Then 0.
#define WRITE_FIELD(TYPE, NAME, DEFAULT, FIELD_NUM) \
writeVarUInt(FIELD_NUM, out); \
writeBinary(NAME, out);
@ -31,7 +31,7 @@ void BlockInfo::write(WriteBuffer & out) const
writeVarUInt(0, out);
}
/// Прочитать значения в бинарном виде.
/// Read values in binary form.
void BlockInfo::read(ReadBuffer & in)
{
UInt64 field_num = 0;

View File

@ -3,11 +3,11 @@ namespace DB
namespace ErrorCodes
{
/** Раньше эти константы были расположены в одном enum-е.
* Но в этом случае возникает проблема: при добавлении новой константы необходимо перекомпилировать
* все translation unit-ы, которые используют хотя бы одну константу (почти весь проект).
* Поэтому сделано так, что определения констант расположены здесь, в одном файле,
* а их объявления - в разных файлах, по месту использования.
/** Previously, these constants were located in one enum.
* But in this case there is a problem: when you add a new constant, you need to recompile
* all translation units that use at least one constant (almost the whole project).
* Therefore it is made so that definitions of constants are located here, in one file,
* and their declaration are in different files, at the place of use.
*/
extern const int UNSUPPORTED_METHOD = 1;

View File

@ -115,7 +115,7 @@ NamesAndTypesList NamesAndTypesList::filter(const Names & names) const
NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
{
/// NOTE Лучше сделать map в IStorage, чем создавать его здесь каждый раз заново.
/// NOTE It's better to make a map in `IStorage` than to create it here every time again.
google::dense_hash_map<StringRef, const DataTypePtr *, StringRefHash> types;
types.set_empty_key(StringRef());

View File

@ -8,13 +8,13 @@ void AddingDefaultBlockOutputStream::write(const DB::Block & block)
{
Block res = block;
/// Вычисляет явно указанные (в column_defaults) значения по-умолчанию.
/// Computes explicitly specified values (in column_defaults) by default.
/** @todo if somehow block does not contain values for implicitly-defaulted columns that are prerequisites
* for explicitly-defaulted ones, exception will be thrown during evaluating such columns
* (implicitly-defaulted columns are evaluated on the line after following one. */
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
/// Добавляет не указанные значения по-умолчанию.
/// Adds not specified default values.
if (!only_explicit_column_defaults)
/// @todo this line may be moved before `evaluateMissingDefaults` with passing {required_columns - explicitly-defaulted columns}
res.addDefaults(*required_columns);

View File

@ -34,15 +34,15 @@ Block AggregatingBlockInputStream::readImpl()
}
else
{
/** Если есть временные файлы с частично-агрегированными данными на диске,
* то читаем и мерджим их, расходуя минимальное количество памяти.
/** If there are temporary files with partially-aggregated data on the disk,
* then read and merge them, spending the minimum amount of memory.
*/
ProfileEvents::increment(ProfileEvents::ExternalAggregationMerge);
if (!isCancelled())
{
/// Сбросим имеющиеся в оперативке данные тоже на диск. Так проще.
/// Flush data in the RAM to disk also. It's easier.
size_t rows = data_variants->sizeWithoutOverflowRow();
if (rows)
aggregator.writeToTemporaryFile(*data_variants, rows);

View File

@ -21,24 +21,24 @@ Block AggregatingSortedBlockInputStream::readImpl()
if (merged_columns.empty())
return Block();
/// Дополнительная инициализация.
/// Additional initialization.
if (next_key.empty())
{
next_key.columns.resize(description.size());
/// Заполним номера столбцов, которые нужно доагрегировать.
/// Fill in the column numbers that need to be aggregated.
for (size_t i = 0; i < num_columns; ++i)
{
ColumnWithTypeAndName & column = merged_block.safeGetByPosition(i);
/// Оставляем только состояния аггрегатных функций.
/// We leave only states of aggregate functions.
if (!startsWith(column.type->getName(), "AggregateFunction"))
{
column_numbers_not_to_aggregate.push_back(i);
continue;
}
/// Входят ли в PK?
/// Included into PK?
SortDescription::const_iterator it = description.begin();
for (; it != description.end(); ++it)
if (it->column_name == column.name || (it->column_name.empty() && it->column_number == i))
@ -72,7 +72,7 @@ void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns,
{
size_t merged_rows = 0;
/// Вынимаем строки в нужном порядке и кладём в merged_block, пока строк не больше max_block_size
/// We take the rows in the correct order and put them in `merged_block`, while the rows are no more than `max_block_size`
while (!queue.empty())
{
TSortCursor current = queue.top();
@ -81,7 +81,7 @@ void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns,
bool key_differs;
if (current_key.empty()) /// Первый встретившийся ключ.
if (current_key.empty()) /// The first key encountered.
{
current_key.columns.resize(description.size());
setPrimaryKeyRef(current_key, current);
@ -90,7 +90,7 @@ void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns,
else
key_differs = next_key != current_key;
/// если накопилось достаточно строк и последняя посчитана полностью
/// if there are enough rows accumulated and the last one is calculated completely
if (key_differs && merged_rows >= max_block_size)
return;
@ -100,14 +100,14 @@ void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns,
{
current_key.swap(next_key);
/// Запишем данные для очередной группы. Копируем значения обычных столбцов.
/// We will write the data for the group. We copy the values of ordinary columns.
for (size_t i = 0, size = column_numbers_not_to_aggregate.size(); i < size; ++i)
{
size_t j = column_numbers_not_to_aggregate[i];
merged_columns[j]->insertFrom(*current->all_columns[j], current->pos);
}
/// Добавляем в агрегатные столбцы пустое состояние агрегации. Состояние будет обновлено в функции addRow.
/// Add the empty aggregation state to the aggregate columns. The state will be updated in the `addRow` function.
for (auto & column_to_aggregate : columns_to_aggregate)
column_to_aggregate->insertDefault();
@ -123,7 +123,7 @@ void AggregatingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns,
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
/// We fetch the next block from the appropriate source, if there is one.
fetchNextBlock(current, queue);
}
}

View File

@ -86,7 +86,7 @@ void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
{
calculated_rows_before_limit = true;
/// есть ли Limit?
/// is there a Limit?
BlockStreamProfileInfos limits;
collectInfosForStreamsWithName("Limit", limits);
@ -94,8 +94,8 @@ void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
{
applied_limit = true;
/** Берём количество строчек, прочитанных ниже PartialSorting-а, если есть, или ниже Limit-а.
* Это нужно, потому что сортировка может вернуть только часть строк.
/** Take the number of lines read below `PartialSorting`, if any, or below `Limit`.
* This is necessary, because sorting can return only part of the rows.
*/
BlockStreamProfileInfos partial_sortings;
collectInfosForStreamsWithName("PartialSorting", partial_sortings);
@ -108,7 +108,7 @@ void BlockStreamProfileInfo::calculateRowsBeforeLimit() const
}
else
{
/// Тогда данные о rows_before_limit могут быть в RemoteBlockInputStream-е (приехать с удалённого сервера).
/// Then the data about `rows_before_limit` can be in `RemoteBlockInputStream` (come from a remote server).
BlockStreamProfileInfos remotes;
collectInfosForStreamsWithName("Remote", remotes);

View File

@ -27,7 +27,7 @@ CSVRowInputStream::CSVRowInputStream(ReadBuffer & istr_, const Block & sample_,
static void skipEndOfLine(ReadBuffer & istr)
{
/// \n (Unix) или \r\n (DOS/Windows) или \n\r (Mac OS Classic)
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
if (*istr.position() == '\n')
{
@ -56,7 +56,7 @@ static void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_
if (istr.eof())
return;
/// поддерживаем лишний разделитель на конце строки
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
{
++istr.position();
@ -71,7 +71,7 @@ static void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_
}
/// Пропустить допустимые в CSV пробельные символы.
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
{
while (!buf.eof()
@ -143,7 +143,7 @@ String CSVRowInputStream::getDiagnosticInfo()
WriteBufferFromString out(res);
Block block = sample.cloneEmpty();
/// Вывести подробную диагностику возможно лишь если последняя и предпоследняя строка ещё находятся в буфере для чтения.
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
@ -161,7 +161,7 @@ String CSVRowInputStream::getDiagnosticInfo()
if (sample.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = sample.safeGetByPosition(i).type->getName().size();
/// Откатываем курсор для чтения на начало предыдущей или текущей строки и парсим всё заново. Но теперь выводим подробную информацию.
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
@ -228,7 +228,7 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
if (data_types[i]->isNumeric())
{
/// Пустая строка вместо числа.
/// An empty string instead of a number.
if (curr_position == prev_position)
{
out << "ERROR: text ";
@ -271,13 +271,13 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
}
}
/// Разделители
/// Delimiters
if (i + 1 == size)
{
if (istr.eof())
return false;
/// поддерживаем лишний разделитель на конце строки
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
{
++istr.position();

View File

@ -1,6 +1,6 @@
#include <DB/DataStreams/CollapsingFinalBlockInputStream.h>
/// Максимальное количество сообщений о некорректных данных в логе.
/// Maximum number of messages about incorrect data in the log.
#define MAX_ERROR_MESSAGES 10
@ -9,7 +9,7 @@ namespace DB
CollapsingFinalBlockInputStream::~CollapsingFinalBlockInputStream()
{
/// Нужно обезвредить все MergingBlockPtr, чтобы они не пытались класть блоки в output_blocks.
/// You must cancel all `MergingBlockPtr` so that they do not try to put blocks in `output_blocks`.
previous.block.cancel();
last_positive.block.cancel();
@ -26,7 +26,7 @@ CollapsingFinalBlockInputStream::~CollapsingFinalBlockInputStream()
void CollapsingFinalBlockInputStream::reportBadCounts()
{
/// При неконсистентных данных, это - неизбежная ошибка, которая не может быть легко исправлена админами. Поэтому Warning.
/// With inconsistent data, this is an unavoidable error that can not be easily fixed by admins. Therefore Warning.
LOG_WARNING(log, "Incorrect data: number of rows with sign = 1 (" << count_positive
<< ") differs with number of rows with sign = -1 (" << count_negative
<< ") by more than one");
@ -82,7 +82,7 @@ Block CollapsingFinalBlockInputStream::readImpl()
first = false;
}
/// Будем формировать блоки для ответа, пока не получится непустой блок.
/// We will create blocks for the answer until we get a non-empty block.
while (true)
{
while (!queue.empty() && output_blocks.empty())
@ -93,7 +93,7 @@ Block CollapsingFinalBlockInputStream::readImpl()
bool has_next = !queue.empty();
Cursor next = has_next ? queue.top() : Cursor();
/// Будем продвигаться в текущем блоке, не используя очередь, пока возможно.
/// We will advance in the current block, not using the queue, as long as possible.
while (true)
{
if (!current.equal(previous))
@ -121,7 +121,7 @@ Block CollapsingFinalBlockInputStream::readImpl()
{
fetchNextBlock(current.block->stream_index);
/// Все потоки кончились. Обработаем последний ключ.
/// All streams are over. We'll process the last key.
if (!has_next)
commitCurrent();
@ -140,7 +140,7 @@ Block CollapsingFinalBlockInputStream::readImpl()
}
}
/// Конец потока.
/// End of the stream.
if (output_blocks.empty())
{
if (blocks_fetched != blocks_output)

View File

@ -2,7 +2,7 @@
#include <DB/DataStreams/CollapsingSortedBlockInputStream.h>
#include <DB/Columns/ColumnsNumber.h>
/// Максимальное количество сообщений о некорректных данных в логе.
/// Maximum number of messages about incorrect data in the log.
#define MAX_ERROR_MESSAGES 10
@ -31,9 +31,9 @@ void CollapsingSortedBlockInputStream::reportIncorrectData()
s << ").";
/** Пока ограничимся всего лишь логгированием таких ситуаций,
* так как данные генерируются внешними программами.
* При неконсистентных данных, это - неизбежная ошибка, которая не может быть легко исправлена админами. Поэтому Warning.
/** Fow now we limit ourselves to just logging such situations,
* since the data is generated by external programs.
* With inconsistent data, this is an unavoidable error that can not be easily corrected by admins. Therefore Warning.
*/
LOG_WARNING(log, s.rdbuf());
}
@ -45,7 +45,7 @@ void CollapsingSortedBlockInputStream::insertRows(ColumnPlainPtrs & merged_colum
{
if (count_positive == count_negative && !last_is_positive)
{
/// Если все строки во входных потоках схлопнулись, мы все равно хотим выдать хоть один блок в результат.
/// If all the rows in the input streams collapsed, we still want to give at least one block in the result.
if (last_in_stream && merged_rows == 0 && !blocks_written)
{
LOG_INFO(log, "All rows collapsed");
@ -111,7 +111,7 @@ Block CollapsingSortedBlockInputStream::readImpl()
if (merged_columns.empty())
return Block();
/// Дополнительная инициализация.
/// Additional initialization.
if (first_negative.empty())
{
first_negative.columns.resize(num_columns);
@ -135,7 +135,7 @@ void CollapsingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, s
{
size_t merged_rows = 0;
/// Вынимаем строки в нужном порядке и кладём в merged_block, пока строк не больше max_block_size
/// Take rows in correct order and put them into `merged_block` until the rows no more than `max_block_size`
for (; !queue.empty(); ++current_pos)
{
TSortCursor current = queue.top();
@ -153,7 +153,7 @@ void CollapsingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, s
bool key_differs = next_key != current_key;
/// если накопилось достаточно строк и последняя посчитана полностью
/// if there are enough rows and the last one is calculated completely
if (key_differs && merged_rows >= max_block_size)
{
++blocks_written;
@ -168,7 +168,7 @@ void CollapsingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, s
if (key_differs)
{
/// Запишем данные для предыдущего первичного ключа.
/// We write data for the previous primary key.
insertRows(merged_columns, merged_rows);
current_key.swap(next_key);
@ -213,12 +213,12 @@ void CollapsingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, s
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
/// We take next block from the corresponding source, if there is one.
fetchNextBlock(current, queue);
}
}
/// Запишем данные для последнего первичного ключа.
/// Write data for last primary key.
insertRows(merged_columns, merged_rows, true);
finished = true;

View File

@ -51,7 +51,7 @@ void CreatingSetsBlockInputStream::createAll()
{
for (auto & elem : subqueries_for_sets)
{
if (elem.second.source) /// Бывают заранее подготовленные Set/Join - для них не указывается source.
if (elem.second.source) /// There could be prepared in advance Set/Join - no source is specified for them.
{
if (isCancelled())
return;
@ -141,11 +141,11 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery)
}
}
// subquery.source->readSuffix(); /// TODO Блокируется в RemoteBlockInputStream::readSuffixImpl при запросе SELECT number FROM system.numbers WHERE number IN (SELECT number FROM remote('127.0.0.{1,2}', system, numbers) WHERE number % 2 = 1 LIMIT 10) LIMIT 10
// subquery.source->readSuffix(); /// TODO Blocked in `RemoteBlockInputStream::readSuffixImpl` when querying `SELECT number FROM system.numbers WHERE number IN (SELECT number FROM remote(`127.0.0.{1,2}', system, numbers) WHERE number % 2 = 1 LIMIT 10) LIMIT 10`
if (table_out)
table_out->writeSuffix();
/// Выведем информацию о том, сколько считано строк и байт.
/// We will display information about how many rows and bytes are read.
size_t rows = 0;
size_t bytes = 0;

View File

@ -61,19 +61,19 @@ Block FilterBlockInputStream::readImpl()
const Block & sample_block = expression->getSampleBlock();
/// Найдём настоящую позицию столбца с фильтром в блоке.
/** sample_block имеет структуру результата вычисления выражения.
* Но эта структура не обязательно совпадает с expression->execute(res) ниже,
* потому что выражение может применяться к блоку, который также содержит дополнительные,
* ненужные для данного выражения столбцы, но нужные позже, в следующих стадиях конвейера выполнения запроса.
* Таких столбцов в sample_block не будет.
* Поэтому, позиция столбца-фильтра в нём может быть другой.
/// Find the current position of the filter column in the block.
/** sample_block has the result structure of evaluating the expression.
* But this structure does not necessarily match expression->execute(res) below,
* because the expression can be applied to a block that also contains additional,
* columns unnecessary for this expression, but needed later, in the next stages of the query execution pipeline.
* There will be no such columns in sample_block.
* Therefore, the position of the filter column in it can be different.
*/
ssize_t filter_column_in_sample_block = filter_column;
if (filter_column_in_sample_block == -1)
filter_column_in_sample_block = sample_block.getPositionByName(filter_column_name);
/// Проверим, не является ли столбец с фильтром константой, содержащей 0 или 1.
/// Let's check if the filter column is a constant containing 0 or 1.
ColumnPtr column = sample_block.safeGetByPosition(filter_column_in_sample_block).column;
if (column)
@ -99,7 +99,7 @@ Block FilterBlockInputStream::readImpl()
return res;
}
/// Пока не встретится блок, после фильтрации которого что-нибудь останется, или поток не закончится.
/// Until the block is encountered, after filtering which something remains, or the stream does not end.
while (1)
{
res = children.back()->read();
@ -111,7 +111,7 @@ Block FilterBlockInputStream::readImpl()
if (filter_always_true)
return res;
/// Найдём настоящую позицию столбца с фильтром в блоке.
/// Find the current position of the filter column in the block.
if (filter_column == -1)
filter_column = res.getPositionByName(filter_column_name);
@ -135,10 +135,10 @@ Block FilterBlockInputStream::readImpl()
const ColumnUInt8 * column_vec = typeid_cast<const ColumnUInt8 *>(observed_column);
if (!column_vec)
{
/** Бывает, что на этапе анализа выражений (в sample_block) столбцы-константы ещё не вычислены,
* а сейчас - вычислены. То есть, не все случаи покрываются кодом выше.
* Это происходит, если функция возвращает константу для неконстантного аргумента.
* Например, функция ignore.
/** It happens that at the stage of analysis of expressions (in sample_block) the columns-constants have not been calculated yet,
* and now - are calculated. That is, not all cases are covered by the code above.
* This happens if the function returns a constant for a non-constant argument.
* For example, `ignore` function.
*/
const ColumnConstUInt8 * column_const = typeid_cast<const ColumnConstUInt8 *>(observed_column);
@ -184,9 +184,9 @@ Block FilterBlockInputStream::readImpl()
const IColumn::Filter & filter = column_vec->getData();
/** Выясним, сколько строк будет в результате.
* Для этого отфильтруем первый попавшийся неконстантный столбец
* или же посчитаем количество выставленных байт в фильтре.
/** Let's find out how many rows will be in result.
* To do this, we filter out the first non-constant column
* or calculate number of set bytes in the filter.
*/
size_t first_non_constant_column = 0;
for (size_t i = 0; i < columns; ++i)
@ -212,27 +212,27 @@ Block FilterBlockInputStream::readImpl()
filtered_rows = countBytesInFilter(filter);
}
/// Если текущий блок полностью отфильтровался - перейдём к следующему.
/// If the current block is completely filtered out, let's move on to the next one.
if (filtered_rows == 0)
continue;
/// Если через фильтр проходят все строчки.
/// If all the rows pass through the filter.
if (filtered_rows == filter.size())
{
/// Заменим столбец с фильтром на константу.
/// Replace the column with the filter by a constant.
res.safeGetByPosition(filter_column).column = std::make_shared<ColumnConstUInt8>(filtered_rows, 1);
/// Остальные столбцы трогать не нужно.
/// No need to touch the rest of the columns.
return res;
}
/// Фильтруем остальные столбцы.
/// Filter the rest of the columns.
for (size_t i = 0; i < columns; ++i)
{
ColumnWithTypeAndName & current_column = res.safeGetByPosition(i);
if (i == static_cast<size_t>(filter_column))
{
/// Сам столбец с фильтром заменяем на столбец с константой 1, так как после фильтрации в нём ничего другого не останется.
/// The column with filter itself is replaced with a column with a constant `1`, since after filtering, nothing else will remain.
current_column.column = std::make_shared<ColumnConstUInt8>(filtered_rows, 1);
continue;
}

View File

@ -185,8 +185,8 @@ static BlockOutputStreamPtr getOutputImpl(const String & name, WriteBuffer & buf
BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & buf,
const Block & sample, const Context & context) const
{
/** Материализация нужна, так как форматы могут использовать функции IDataType,
* которые допускают работу только с полными столбцами.
/** Materialization is needed, because formats can use the functions `IDataType`,
* which only work with full columns.
*/
return std::make_shared<MaterializingBlockOutputStream>(getOutputImpl(name, buf, sample, context));
}

View File

@ -36,16 +36,16 @@ UInt32 GraphiteRollupSortedBlockInputStream::selectPrecision(const Graphite::Ret
}
/** Округлить unix timestamp до precision секунд.
* При этом, дата не должна измениться. Дата исчисляется с помощью локального часового пояса.
/** Round the unix timestamp to seconds precision.
* In this case, the date should not change. The date is calculated using the local time zone.
*
* Если величина округления не больше часа,
* то, исходя из допущения, что часовые пояса, отличающиеся от UTC на нецелое количество часов не поддерживаются,
* достаточно просто округлить unix timestamp вниз до числа, кратного 3600.
* А если величина округления больше,
* то будем подвергать округлению число секунд от начала суток в локальном часовом поясе.
* If the rounding value is less than an hour,
* then, assuming that time zones that differ from UTC by a non-integer number of hours are not supported,
* just simply round the unix timestamp down to a multiple of 3600.
* And if the rounding value is greater,
* then we will round down the number of seconds from the beginning of the day in the local time zone.
*
* Округление более чем до суток не поддерживается.
* Rounding to more than a day is not supported.
*/
static time_t roundTimeToPrecision(const DateLUTImpl & date_lut, time_t time, UInt32 precision)
{
@ -114,7 +114,7 @@ void GraphiteRollupSortedBlockInputStream::merge(ColumnPlainPtrs & merged_column
size_t merged_rows = 0;
/// Вынимаем строки в нужном порядке и кладём в merged_block, пока строк не больше max_block_size
/// Take rows in needed order and put them into `merged_block` until rows no more than `max_block_size`
while (!queue.empty())
{
TSortCursor current = queue.top();
@ -195,7 +195,7 @@ void GraphiteRollupSortedBlockInputStream::merge(ColumnPlainPtrs & merged_column
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
/// We get the next block from the appropriate source, if there is one.
fetchNextBlock(current, queue);
}
}
@ -218,7 +218,7 @@ void GraphiteRollupSortedBlockInputStream::merge(ColumnPlainPtrs & merged_column
template <class TSortCursor>
void GraphiteRollupSortedBlockInputStream::startNextRow(ColumnPlainPtrs & merged_columns, TSortCursor & cursor)
{
/// Копируем не модифицированные значения столбцов.
/// Copy unmodified column values.
for (size_t i = 0, size = unmodified_column_numbers.size(); i < size; ++i)
{
size_t j = unmodified_column_numbers[i];
@ -232,7 +232,7 @@ void GraphiteRollupSortedBlockInputStream::startNextRow(ColumnPlainPtrs & merged
void GraphiteRollupSortedBlockInputStream::finishCurrentRow(ColumnPlainPtrs & merged_columns)
{
/// Вставляем вычисленные значения столбцов time, value, version.
/// Insert calculated values of the columns `time`, `value`, `version`.
merged_columns[time_column_num]->insert(UInt64(current_time_rounded));
merged_columns[version_column_num]->insert(current_max_version);

View File

@ -61,7 +61,7 @@ size_t IBlockInputStream::checkDepthImpl(size_t max_depth, size_t level) const
void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t multiplier)
{
/// Не будем отображать в дереве обёртку потока блоков в AsynchronousBlockInputStream.
/// We will not display the wrapper of the block stream in the AsynchronousBlockInputStream in the tree.
if (getName() != "Asynchronous")
{
ostr << String(indent, ' ') << getName();
@ -70,7 +70,7 @@ void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t mult
ostr << std::endl;
++indent;
/// Если поддерево повторяется несколько раз, то будем выводить его один раз с множителем.
/// If the subtree is repeated several times, then we output it once with the multiplier.
using Multipliers = std::map<String, size_t>;
Multipliers multipliers;
@ -81,7 +81,7 @@ void IBlockInputStream::dumpTree(std::ostream & ostr, size_t indent, size_t mult
{
String id = (*it)->getTreeID();
size_t & subtree_multiplier = multipliers[id];
if (subtree_multiplier != 0) /// Уже выведенные поддеревья помечаем нулём в массиве множителей.
if (subtree_multiplier != 0) /// Already printed subtrees are marked with zero in the array of multipliers.
{
(*it)->dumpTree(ostr, indent, subtree_multiplier);
subtree_multiplier = 0;

View File

@ -31,7 +31,7 @@ Block IProfilingBlockInputStream::read()
if (const IProfilingBlockInputStream * p_child = dynamic_cast<const IProfilingBlockInputStream *>(&*child))
info.nested_infos.push_back(&p_child->info);
/// Заметим, что после такого, элементы children нельзя удалять до того, как может потребоваться работать с nested_info.
/// Note that after this, `children` elements can not be deleted before you might need to work with `nested_info`.
info.started = true;
}
@ -59,11 +59,11 @@ Block IProfilingBlockInputStream::read()
}
else
{
/** Если поток закончился, то ещё попросим всех детей прервать выполнение.
* Это имеет смысл при выполнении запроса с LIMIT-ом:
* - бывает ситуация, когда все необходимые данные уже прочитали,
* но источники-дети ещё продолжают работать,
* при чём они могут работать в отдельных потоках или даже удалённо.
/** If the thread is over, then we will ask all children to abort the execution.
* This makes sense when running a query with LIMIT
* - there is a situation when all the necessary data has already been read,
* but `children sources are still working,
* herewith they can work in separate threads or even remotely.
*/
cancel();
}
@ -201,7 +201,7 @@ void IProfilingBlockInputStream::checkQuota(Block & block)
switch (limits.mode)
{
case LIMITS_TOTAL:
/// Проверяется в методе progress.
/// Checked in `progress` method.
break;
case LIMITS_CURRENT:
@ -232,15 +232,15 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
if (!process_list_elem->updateProgressIn(value))
cancel();
/// Общее количество данных, обработанных или предполагаемых к обработке во всех листовых источниках, возможно, на удалённых серверах.
/// The total amount of data processed or intended for processing in all leaf sources, possibly on remote servers.
size_t rows_processed = process_list_elem->progress_in.rows;
size_t bytes_processed = process_list_elem->progress_in.bytes;
size_t total_rows_estimate = std::max(rows_processed, process_list_elem->progress_in.total_rows.load(std::memory_order_relaxed));
/** Проверяем ограничения на объём данных для чтения, скорость выполнения запроса, квоту на объём данных для чтения.
* NOTE: Может быть, имеет смысл сделать, чтобы они проверялись прямо в ProcessList?
/** Check the restrictions on the amount of data to read, the speed of the query, the quota on the amount of data to read.
* NOTE: Maybe it makes sense to have them checked directly in ProcessList?
*/
if (limits.mode == LIMITS_TOTAL
@ -260,7 +260,7 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
}
else if (limits.read_overflow_mode == OverflowMode::BREAK)
{
/// Для break будем останавливаться только если действительно было прочитано столько строк, а не только предполагается к чтению.
/// For `break`, we will stop only if so many lines were actually read, and not just supposed to be read.
if ((limits.max_rows_to_read && rows_processed > limits.max_rows_to_read)
|| (limits.max_bytes_to_read && bytes_processed > limits.max_bytes_to_read))
{
@ -286,7 +286,7 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value)
size_t total_rows = process_list_elem->progress_in.total_rows;
/// Если предсказанное время выполнения больше, чем max_execution_time.
/// If the predicted execution time is longer than `max_execution_time`.
if (limits.max_execution_time != 0 && total_rows)
{
double estimated_execution_time_seconds = total_elapsed * (static_cast<double>(total_rows) / rows_processed);

View File

@ -20,13 +20,13 @@ JSONEachRowRowInputStream::JSONEachRowRowInputStream(ReadBuffer & istr_, const B
size_t columns = sample.columns();
for (size_t i = 0; i < columns; ++i)
name_map[sample.safeGetByPosition(i).name] = i; /// NOTE Можно было бы расположить имена более кэш-локально.
name_map[sample.safeGetByPosition(i).name] = i; /// NOTE You could place names more cache-locally.
}
/** Прочитать имя поля в формате JSON.
* Ссылка на имя поля будет записана в ref.
* Также может быть использован временный буфер tmp, чтобы скопировать туда имя поля.
/** Read the field name in JSON format.
* A reference to the field name will be written to ref.
* You can also use temporary `tmp` buffer to copy field name there.
*/
static StringRef readName(ReadBuffer & buf, String & tmp)
{
@ -36,7 +36,7 @@ static StringRef readName(ReadBuffer & buf, String & tmp)
if (next_pos != buf.buffer().end() && *next_pos != '\\')
{
/// Наиболее вероятный вариант - в имени ключа нет эскейп-последовательностей и имя целиком поместилось в буфер.
/// The most likely option is that there is no escape sequence in the key name, and the entire name is placed in the buffer.
assertChar('"', buf);
StringRef res(buf.position(), next_pos - buf.position());
buf.position() += next_pos - buf.position();
@ -68,8 +68,8 @@ bool JSONEachRowRowInputStream::read(Block & block)
size_t columns = block.columns();
/// Множество столбцов, для которых были считаны значения. Остальные затем заполним значениями по-умолчанию.
/// TODO Возможность предоставить свои DEFAULT-ы.
/// Set of columns for which the values were read. The rest will be filled with default values.
/// TODO Ability to provide your DEFAULTs.
bool read_columns[columns];
memset(read_columns, 0, columns);
@ -96,8 +96,8 @@ bool JSONEachRowRowInputStream::read(Block & block)
StringRef name_ref = readName(istr, name_buf);
/// NOTE Возможна оптимизация путём кэширования порядка полей (который почти всегда одинаковый)
/// и быстрой проверки на соответствие следующему ожидаемому полю, вместо поиска в хэш-таблице.
/// NOTE Optimization is possible by caching the order of fields (which is almost always the same)
/// and a quick check to match the next expected field, instead of searching the hash table.
auto it = name_map.find(name_ref);
if (name_map.end() == it)
@ -127,7 +127,7 @@ bool JSONEachRowRowInputStream::read(Block & block)
if (!istr.eof() && *istr.position() == ',')
++istr.position();
/// Заполняем не встретившиеся столбцы значениями по-умолчанию.
/// Fill non-visited columns with the default values.
for (size_t i = 0; i < columns; ++i)
if (!read_columns[i])
block.getByPosition(i).column.get()->insertDefault();

View File

@ -18,7 +18,7 @@ Block LimitBlockInputStream::readImpl()
Block res;
size_t rows = 0;
/// pos - сколько строк было прочитано, включая последний прочитанный блок
/// pos - how many lines were read, including the last read block
if (pos >= offset + limit)
{
@ -41,11 +41,11 @@ Block LimitBlockInputStream::readImpl()
pos += rows;
} while (pos <= offset);
/// отдать целый блок
/// give away the whole block
if (pos >= offset + rows && pos <= offset + limit)
return res;
/// отдать кусок блока
/// give away a piece of the block
size_t start = std::max(
static_cast<Int64>(0),
static_cast<Int64>(offset) - static_cast<Int64>(pos) + static_cast<Int64>(rows));

View File

@ -32,7 +32,7 @@ String MergingSortedBlockInputStream::getID() const
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Порядок не имеет значения.
/// The order does not matter.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
@ -47,7 +47,7 @@ String MergingSortedBlockInputStream::getID() const
void MergingSortedBlockInputStream::init(Block & merged_block, ColumnPlainPtrs & merged_columns)
{
/// Читаем первые блоки, инициализируем очередь.
/// Read the first blocks, initialize the queue.
if (first)
{
first = false;
@ -83,9 +83,9 @@ void MergingSortedBlockInputStream::init(Block & merged_block, ColumnPlainPtrs &
initQueue(queue);
}
/// Инициализируем результат.
/// Initialize the result.
/// Клонируем структуру первого непустого блока источников.
/// We clone the structure of the first non-empty source block.
{
auto it = source_blocks.cbegin();
for (; it != source_blocks.cend(); ++it)
@ -99,12 +99,12 @@ void MergingSortedBlockInputStream::init(Block & merged_block, ColumnPlainPtrs &
}
}
/// Если все входные блоки пустые.
/// If all the input blocks are empty.
if (it == source_blocks.cend())
return;
}
/// Проверим, что у всех блоков-источников одинаковая структура.
/// Let's check that all source blocks have the same structure.
for (auto it = source_blocks.cbegin(); it != source_blocks.cend(); ++it)
{
const SharedBlockPtr & shared_block_ptr = *it;
@ -178,8 +178,8 @@ void MergingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs
{
size_t merged_rows = 0;
/** Увеличить счётчики строк.
* Вернуть true, если пора закончить формировать текущий блок данных.
/** Increase row counters.
* Return true if it's time to finish generating the current data block.
*/
auto count_row_and_check_limit = [&, this]()
{
@ -202,7 +202,7 @@ void MergingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs
return false;
};
/// Вынимаем строки в нужном порядке и кладём в merged_block, пока строк не больше max_block_size
/// Take rows in required order and put them into `merged_block`, while the rows are no more than `max_block_size`
while (!queue.empty())
{
TSortCursor current = queue.top();
@ -210,14 +210,14 @@ void MergingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs
while (true)
{
/** А вдруг для текущего курсора блок целиком меньше или равен, чем остальные?
* Или в очереди остался только один источник данных? Тогда можно целиком взять блок текущего курсора.
/** And what if the block is smaller or equal than the rest for the current cursor?
* Or is there only one data source left in the queue? Then you can take the entire block of current cursor.
*/
if (current.impl->isFirst() && (queue.empty() || current.totallyLessOrEquals(queue.top())))
{
// std::cerr << "current block is totally less or equals\n";
/// Если в текущем блоке уже есть данные, то сначала вернём его. Мы попадём сюда снова при следующем вызове функции merge.
/// If there are already data in the current block, we first return it. We'll get here again the next time we call the merge function.
if (merged_rows != 0)
{
// std::cerr << "merged rows is non-zero\n";
@ -286,7 +286,7 @@ void MergingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs
return;
}
/// Не кладём курсор обратно в очередь, а продолжаем работать с текущим курсором.
/// Do not put the cursor back in the queue, but continue to work with the current cursor.
// std::cerr << "current is still on top, using current row\n";
continue;
}
@ -298,7 +298,7 @@ void MergingSortedBlockInputStream::merge(Block & merged_block, ColumnPlainPtrs
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
/// We get the next block from the corresponding source, if there is one.
// std::cerr << "It was last row, fetching next block\n";
fetchNextBlock(current, queue);
}

View File

@ -63,7 +63,7 @@ void NativeBlockInputStream::readData(const IDataType & type, IColumn & column,
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/** Для массивов требуется сначала десериализовать смещения, а потом значения.
/** For arrays, you first need to deserialize the offsets, and then the values.
*/
IColumn & offsets_column = *typeid_cast<ColumnArray &>(column).getOffsetsColumn();
type_arr->getOffsetsType()->deserializeBinaryBulk(offsets_column, istr, rows, 0);
@ -79,7 +79,7 @@ void NativeBlockInputStream::readData(const IDataType & type, IColumn & column,
typeid_cast<const ColumnArray &>(column).getOffsets()[rows - 1]);
}
else
type.deserializeBinaryBulk(column, istr, rows, 0); /// TODO Использовать avg_value_size_hint.
type.deserializeBinaryBulk(column, istr, rows, 0); /// TODO Use avg_value_size_hint.
if (column.size() != rows)
throw Exception("Cannot read all data in NativeBlockInputStream.", ErrorCodes::CANNOT_READ_ALL_DATA);
@ -103,11 +103,11 @@ Block NativeBlockInputStream::readImpl()
return res;
}
/// Дополнительная информация о блоке.
/// Additional information about the block.
if (server_revision >= DBMS_MIN_REVISION_WITH_BLOCK_INFO)
res.info.read(istr);
/// Размеры
/// Dimensions
size_t columns = 0;
size_t rows = 0;
@ -126,7 +126,7 @@ Block NativeBlockInputStream::readImpl()
{
if (use_index)
{
/// Если текущая позиция и так какая требуется, то реального seek-а не происходит.
/// If the current position is what is required, the real seek does not occur.
istr_concrete->seek(index_column_it->location.offset_in_compressed_file, index_column_it->location.offset_in_decompressed_block);
}

View File

@ -44,8 +44,8 @@ void NativeBlockOutputStream::flush()
void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit)
{
/** Если есть столбцы-константы - то материализуем их.
* (Так как тип данных не умеет сериализовывать/десериализовывать константы.)
/** If there are columns-constants - then we materialize them.
* (Since the data type does not know how to serialize / deserialize constants.)
*/
ColumnPtr full_column;
@ -69,7 +69,7 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
}
else if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
/** Для массивов требуется сначала сериализовать смещения, а потом значения.
/** For arrays, you first need to serialize the offsets, and then the values.
*/
const ColumnArray & column_array = typeid_cast<const ColumnArray &>(*full_column);
type_arr->getOffsetsType()->serializeBinaryBulk(*column_array.getOffsetsColumn(), ostr, offset, limit);
@ -81,12 +81,12 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
if (offset > offsets.size())
return;
/** offset - с какого массива писать.
* limit - сколько массивов максимум записать, или 0, если писать всё, что есть.
* end - до какого массива заканчивается записываемый кусок.
/** offset - from which array to write.
* limit - how many arrays should be written, or 0, if you write everything that is.
* end - up to which array written part finishes.
*
* nested_offset - с какого элемента внутренностей писать.
* nested_limit - сколько элементов внутренностей писать, или 0, если писать всё, что есть.
* nested_offset - from which nested element to write.
* nested_limit - how many nested elements to write, or 0, if you write everything that is.
*/
size_t end = std::min(offset + limit, offsets.size());
@ -118,19 +118,19 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr
void NativeBlockOutputStream::write(const Block & block)
{
/// Дополнительная информация о блоке.
/// Additional information about the block.
if (client_revision >= DBMS_MIN_REVISION_WITH_BLOCK_INFO)
block.info.write(ostr);
/// Размеры
/// Dimensions
size_t columns = block.columns();
size_t rows = block.rows();
writeVarUInt(columns, ostr);
writeVarUInt(rows, ostr);
/** Индекс имеет ту же структуру, что и поток с данными.
* Но вместо значений столбца он содержит засечку, ссылающуюся на место в файле с данными, где находится этот кусочек столбца.
/** The index has the same structure as the data stream.
* But instead of column values, it contains a mark that points to the location in the data file where this part of the column is located.
*/
if (index_ostr)
{
@ -140,12 +140,12 @@ void NativeBlockOutputStream::write(const Block & block)
for (size_t i = 0; i < columns; ++i)
{
/// Для индекса.
/// For the index.
MarkInCompressedFile mark;
if (index_ostr)
{
ostr_concrete->next(); /// Заканчиваем сжатый блок.
ostr_concrete->next(); /// Finish compressed block.
mark.offset_in_compressed_file = initial_size_of_file + ostr_concrete->getCompressedBytes();
mark.offset_in_decompressed_block = ostr_concrete->getRemainingBytes();
}

View File

@ -38,7 +38,7 @@ String ParallelAggregatingBlockInputStream::getID() const
for (size_t i = 0; i < children.size(); ++i)
children_ids[i] = children[i]->getID();
/// Порядок не имеет значения.
/// Order does not matter.
std::sort(children_ids.begin(), children_ids.end());
for (size_t i = 0; i < children_ids.size(); ++i)
@ -74,14 +74,14 @@ Block ParallelAggregatingBlockInputStream::readImpl()
if (!aggregator.hasTemporaryFiles())
{
/** Если все частично-агрегированные данные в оперативке, то мерджим их параллельно, тоже в оперативке.
/** If all partially-aggregated data is in RAM, then merge them in parallel, also in RAM.
*/
impl = aggregator.mergeAndConvertToBlocks(many_data, final, max_threads);
}
else
{
/** Если есть временные файлы с частично-агрегированными данными на диске,
* то читаем и мерджим их, расходуя минимальное количество памяти.
/** If there are temporary files with partially-aggregated data on the disk,
* then read and merge them, spending the minimum amount of memory.
*/
ProfileEvents::increment(ProfileEvents::ExternalAggregationMerge);
@ -133,7 +133,7 @@ void ParallelAggregatingBlockInputStream::Handler::onFinishThread(size_t thread_
{
if (!parent.isCancelled() && parent.aggregator.hasTemporaryFiles())
{
/// Сбросим имеющиеся в оперативке данные тоже на диск. Так проще их потом объединять.
/// Flush data in the RAM to disk. So it's easier to unite them later.
auto & data = *parent.many_data[thread_num];
if (data.isConvertibleToTwoLevel())
@ -149,8 +149,8 @@ void ParallelAggregatingBlockInputStream::Handler::onFinish()
{
if (!parent.isCancelled() && parent.aggregator.hasTemporaryFiles())
{
/// Может так получиться, что какие-то данные ещё не сброшены на диск,
/// потому что во время вызова onFinishThread ещё никакие данные не были сброшены на диск, а потом какие-то - были.
/// It may happen that some data has not yet been flushed,
/// because at the time of `onFinishThread` call, no data has been flushed to disk, and then some were.
for (auto & data : parent.many_data)
{
if (data->isConvertibleToTwoLevel())

View File

@ -101,7 +101,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
return;
}
/// Будем вставлять суда столбцы с вычисленными значениями видимых длин.
/// We will insert here columns with the calculated values of visible lengths.
Block block = block_;
size_t rows = block.rows();
@ -111,7 +111,7 @@ void PrettyBlockOutputStream::write(const Block & block_)
Widths_t name_widths;
calculateWidths(block, max_widths, name_widths);
/// Создадим разделители
/// Create separators
std::stringstream top_separator;
std::stringstream middle_names_separator;
std::stringstream middle_values_separator;
@ -149,10 +149,10 @@ void PrettyBlockOutputStream::write(const Block & block_)
std::string middle_values_separator_s = middle_values_separator.str();
std::string bottom_separator_s = bottom_separator.str();
/// Выведем блок
/// Output the block
writeString(top_separator_s, ostr);
/// Имена
/// Names
writeCString("", ostr);
for (size_t i = 0; i < columns; ++i)
{

View File

@ -18,7 +18,7 @@ void PrettyCompactBlockOutputStream::writeHeader(
const Widths_t & max_widths,
const Widths_t & name_widths)
{
/// Имена
/// Names
writeCString("┌─", ostr);
for (size_t i = 0; i < max_widths.size(); ++i)
{
@ -55,7 +55,7 @@ void PrettyCompactBlockOutputStream::writeHeader(
void PrettyCompactBlockOutputStream::writeBottom(const Widths_t & max_widths)
{
/// Создадим разделители
/// Create delimiters
std::stringstream bottom_separator;
bottom_separator << "";
@ -118,7 +118,7 @@ void PrettyCompactBlockOutputStream::write(const Block & block_)
return;
}
/// Будем вставлять сюда столбцы с вычисленными значениями видимых длин.
/// We will insert columns here with the calculated values of visible lengths.
Block block = block_;
size_t rows = block.rows();

View File

@ -15,7 +15,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
return;
}
/// Будем вставлять суда столбцы с вычисленными значениями видимых длин.
/// We will insert here columns with the calculated values of visible lengths.
Block block = block_;
size_t rows = block.rows();
@ -25,13 +25,13 @@ void PrettySpaceBlockOutputStream::write(const Block & block_)
Widths_t name_widths;
calculateWidths(block, max_widths, name_widths);
/// Не будем выравнивать по слишком длинным значениям.
/// Do not align on too long values.
if (terminal_width > 80)
for (size_t i = 0; i < columns; ++i)
if (max_widths[i] > terminal_width / 2)
max_widths[i] = terminal_width / 2;
/// Имена
/// Names
for (size_t i = 0; i < columns; ++i)
{
if (i != 0)

View File

@ -54,9 +54,9 @@ RemoteBlockInputStream::RemoteBlockInputStream(ConnectionPoolsPtr & pools_, cons
RemoteBlockInputStream::~RemoteBlockInputStream()
{
/** Если прервались в середине цикла общения с репликами, то прервываем
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
* эти соединения не остались висеть в рассихронизированном состоянии.
/** If interrupted in the middle of the loop of communication with replicas, then interrupt
* all connections, then read and skip the remaining packets to make sure
* these connections did not remain hanging in the out-of-sync state.
*/
if (established || isQueryPending())
multiplexed_connections->disconnect();
@ -87,7 +87,7 @@ void RemoteBlockInputStream::cancel()
{
std::lock_guard<std::mutex> lock(external_tables_mutex);
/// Останавливаем отправку внешних данных.
/// Stop sending external data.
for (auto & vec : external_tables_data)
for (auto & elem : vec)
if (IProfilingBlockInputStream * stream = dynamic_cast<IProfilingBlockInputStream *>(elem.first.get()))
@ -150,10 +150,10 @@ Block RemoteBlockInputStream::readImpl()
switch (packet.type)
{
case Protocol::Server::Data:
/// Если блок не пуст и не является заголовочным блоком
/// If the block is not empty and is not a header block
if (packet.block && (packet.block.rows() > 0))
return packet.block;
break; /// Если блок пуст - получим другие пакеты до EndOfStream.
break; /// If the block is empty - we will receive other packets before EndOfStream.
case Protocol::Server::Exception:
got_exception_from_replica = true;
@ -169,11 +169,11 @@ Block RemoteBlockInputStream::readImpl()
break;
case Protocol::Server::Progress:
/** Используем прогресс с удалённого сервера.
* В том числе, запишем его в ProcessList,
* и будем использовать его для проверки
* ограничений (например, минимальная скорость выполнения запроса)
* и квот (например, на количество строчек для чтения).
/** We use the progress from a remote server.
* We also include in ProcessList,
* and we use it to check
* constraints (for example, the minimum speed of query execution)
* and quotas (for example, the number of lines to read).
*/
progressImpl(packet.progress);
break;
@ -200,24 +200,25 @@ Block RemoteBlockInputStream::readImpl()
void RemoteBlockInputStream::readSuffixImpl()
{
/** Если одно из:
* - ничего не начинали делать;
* - получили все пакеты до EndOfStream;
/** If one of:
* - nothing started to do;
* - received all packets before EndOfStream;
* - получили с одной реплики эксепшен;
* - получили с одной реплики неизвестный пакет;
* то больше читать ничего не нужно.
* - received exception from one replica;
* - received an unknown packet from one replica;
* then you do not need to read anything.
*/
if (!isQueryPending() || hasThrownException())
return;
/** Если ещё прочитали не все данные, но они больше не нужны.
* Это может быть из-за того, что данных достаточно (например, при использовании LIMIT).
/** If you have not read all the data yet, but they are no longer needed.
* This may be due to the fact that the data is sufficient (for example, when using LIMIT).
*/
/// Отправим просьбу прервать выполнение запроса, если ещё не отправляли.
/// Send the request to abort the execution of the request, if not already sent.
tryCancel("Cancelling query because enough data has been read");
/// Получим оставшиеся пакеты, чтобы не было рассинхронизации в соединениях с репликами.
/// Get the remaining packages so that there is no out of sync in the connections to the replicas.
Connection::Packet packet = multiplexed_connections->drain();
switch (packet.type)
{

View File

@ -52,7 +52,7 @@ void ReplacingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, st
{
size_t merged_rows = 0;
/// Вынимаем строки в нужном порядке и кладём в merged_block, пока строк не больше max_block_size
/// Take the rows in needed order and put them into `merged_block` until rows no more than `max_block_size`
while (!queue.empty())
{
TSortCursor current = queue.top();
@ -73,7 +73,7 @@ void ReplacingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, st
bool key_differs = next_key != current_key;
/// если накопилось достаточно строк и последняя посчитана полностью
/// if there are enough rows and the last one is calculated completely
if (key_differs && merged_rows >= max_block_size)
return;
@ -82,12 +82,12 @@ void ReplacingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, st
if (key_differs)
{
max_version = 0;
/// Запишем данные для предыдущего первичного ключа.
/// Write the data for the previous primary key.
insertRow(merged_columns, merged_rows);
current_key.swap(next_key);
}
/// Нестрогое сравнение, так как мы выбираем последнюю строку для одинаковых значений версий.
/// A non-strict comparison, since we select the last row for the same version values.
if (version >= max_version)
{
max_version = version;
@ -101,12 +101,12 @@ void ReplacingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, st
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
/// We get the next block from the corresponding source, if there is one.
fetchNextBlock(current, queue);
}
}
/// Запишем данные для последнего первичного ключа.
/// We will write the data for the last primary key.
insertRow(merged_columns, merged_rows);
finished = true;

View File

@ -63,19 +63,19 @@ Block SummingSortedBlockInputStream::readImpl()
if (merged_columns.empty())
return Block();
/// Дополнительная инициализация.
/// Additional initialization.
if (current_row.empty())
{
current_row.resize(num_columns);
next_key.columns.resize(description.size());
/// Имя вложенной структуры -> номера столбцов, которые к ней относятся.
/// name of nested structure -> the column numbers that refer to it.
std::unordered_map<std::string, std::vector<size_t>> discovered_maps;
/** Заполним номера столбцов, которые должны быть просуммированы.
* Это могут быть только числовые столбцы, не входящие в ключ сортировки.
* Если задан непустой список column_names_to_sum, то берём только эти столбцы.
* Часть столбцов из column_names_to_sum может быть не найдена. Это игнорируется.
/** Fill in the column numbers, which must be summed.
* This can only be numeric columns that are not part of the sort key.
* If a non-empty column_names_to_sum is specified, then we only take these columns.
* Some columns from column_names_to_sum may not be found. This is ignored.
*/
for (size_t i = 0; i < num_columns; ++i)
{
@ -93,7 +93,7 @@ Block SummingSortedBlockInputStream::readImpl()
}
else
{
/// Оставляем только числовые типы. При чём, даты и даты-со-временем здесь такими не считаются.
/// Leave only numeric types. Note that dates and datetime here are not considered such.
if (!column.type->isNumeric() ||
column.type->getName() == "Date" ||
column.type->getName() == "DateTime" ||
@ -101,7 +101,7 @@ Block SummingSortedBlockInputStream::readImpl()
column.type->getName() == "Nullable(DateTime)")
continue;
/// Входят ли в PK?
/// Do they enter the PK?
if (isInPrimaryKey(description, column.name, i))
continue;
@ -180,7 +180,7 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
{
size_t merged_rows = 0;
/// Вынимаем строки в нужном порядке и кладём в merged_block, пока строк не больше max_block_size
/// Take the rows in needed order and put them in `merged_block` until rows no more than `max_block_size`
while (!queue.empty())
{
TSortCursor current = queue.top();
@ -189,8 +189,8 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
bool key_differs;
if (current_key.empty()) /// Первый встретившийся ключ.
{
if (current_key.empty()) /// The first key encountered.
{
current_key.columns.resize(description.size());
setPrimaryKeyRef(current_key, current);
key_differs = true;
@ -198,7 +198,7 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
else
key_differs = next_key != current_key;
/// если накопилось достаточно строк и последняя посчитана полностью
/// if there are enough rows and the last one is calculated completely
if (key_differs && merged_rows >= max_block_size)
return;
@ -206,7 +206,7 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
if (key_differs)
{
/// Запишем данные для предыдущей группы.
/// Write the data for the previous group.
if (!current_row_is_zero)
{
++merged_rows;
@ -231,13 +231,13 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
}
else
{
/// Достаём из соответствующего источника следующий блок, если есть.
/// We get the next block from the corresponding source, if there is one.
fetchNextBlock(current, queue);
}
}
/// Запишем данные для последней группы, если она ненулевая.
/// Если она нулевая, и без нее выходной поток окажется пустым, запишем ее все равно.
/// We will write the data for the last group, if it is non-zero.
/// If it is zero, and without it the output stream will be empty, we will write it anyway.
if (!current_row_is_zero || !output_is_non_empty)
{
++merged_rows;
@ -248,8 +248,8 @@ void SummingSortedBlockInputStream::merge(ColumnPlainPtrs & merged_columns, std:
}
/** Реализует операцию +=.
* Возвращает false, если результат получился нулевым.
/** Implements `+=` operation.
* Returns false if the result is zero.
*/
class FieldVisitorSum : public StaticVisitor<bool>
{
@ -285,7 +285,7 @@ bool SummingSortedBlockInputStream::mergeMaps(Row & row, TSortCursor & cursor)
template <class TSortCursor>
bool SummingSortedBlockInputStream::mergeMap(const MapDescription & desc, Row & row, TSortCursor & cursor)
{
/// Сильно неоптимально.
/// Strongly non-optimal.
Row & left = row;
Row right(left.size());
@ -369,7 +369,7 @@ bool SummingSortedBlockInputStream::mergeMap(const MapDescription & desc, Row &
template <class TSortCursor>
bool SummingSortedBlockInputStream::addRow(Row & row, TSortCursor & cursor)
{
bool res = mergeMaps(row, cursor); /// Есть ли хотя бы одно ненулевое число или непустой массив
bool res = mergeMaps(row, cursor); /// Is there at least one non-zero number or non-empty array
for (size_t i = 0, size = column_numbers_to_sum.size(); i < size; ++i)
{

View File

@ -22,16 +22,16 @@ TSKVRowInputStream::TSKVRowInputStream(ReadBuffer & istr_, const Block & sample_
size_t columns = sample.columns();
for (size_t i = 0; i < columns; ++i)
name_map[sample.safeGetByPosition(i).name] = i; /// NOTE Можно было бы расположить имена более кэш-локально.
name_map[sample.safeGetByPosition(i).name] = i; /// NOTE You could place names more cache-locally.
}
/** Прочитать имя поля в формате tskv.
* Вернуть true, если после имени поля идёт знак равенства,
* иначе (поле без значения) вернуть false.
* Ссылка на имя поля будет записана в ref.
* Также может быть использован временный буфер tmp, чтобы скопировать туда имя поля.
* При чтении, пропускает имя и знак равенства после него.
/** Read the field name in the `tskv` format.
* Return true if the field is followed by an equal sign,
* otherwise (field with no value) return false.
* The reference to the field name will be written to `ref`.
* A temporary `tmp` buffer can also be used to copy the field name to it.
* When reading, skips the name and the equal sign after it.
*/
static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp)
{
@ -48,26 +48,26 @@ static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp)
continue;
}
/// Дошли до конца имени.
/// Came to the end of the name.
if (*next_pos != '\\')
{
bool have_value = *next_pos == '=';
if (tmp.empty())
{
/// Данные не нужно копировать, можно ссылаться прямо на внутренность buf.
/// No need to copy data, you can refer directly to the `buf`.
ref = StringRef(buf.position(), next_pos - buf.position());
buf.position() += next_pos + have_value - buf.position();
}
else
{
/// Копируем данные во временную строку и возвращаем ссылку на неё.
/// Copy the data to a temporary string and return a reference to it.
tmp.append(buf.position(), next_pos - buf.position());
buf.position() += next_pos + have_value - buf.position();
ref = StringRef(tmp);
}
return have_value;
}
/// В имени есть эскейп-последовательность.
/// The name has an escape sequence.
else
{
tmp.append(buf.position(), next_pos - buf.position());
@ -92,14 +92,14 @@ bool TSKVRowInputStream::read(Block & block)
size_t columns = block.columns();
/// Множество столбцов, для которых были считаны значения. Остальные затем заполним значениями по-умолчанию.
/// TODO Возможность предоставить свои DEFAULT-ы.
/// Set of columns for which the values were read. The rest will be filled with default values.
/// TODO Ability to provide your DEFAULTs.
bool read_columns[columns];
memset(read_columns, 0, columns);
if (unlikely(*istr.position() == '\n'))
{
/// Пустая строка. Допустимо, но непонятно зачем.
/// An empty string. It is permissible, but it is unclear why.
++istr.position();
}
else
@ -111,8 +111,8 @@ bool TSKVRowInputStream::read(Block & block)
if (has_value)
{
/// NOTE Возможна оптимизация путём кэширования порядка полей (который почти всегда одинаковый)
/// и быстрой проверки на соответствие следующему ожидаемому полю, вместо поиска в хэш-таблице.
/// NOTE Optimization is possible by caching the order of fields (which is almost always the same)
/// and quickly checking for the next expected field, instead of searching the hash table.
auto it = name_map.find(name_ref);
if (name_map.end() == it)
@ -120,7 +120,7 @@ bool TSKVRowInputStream::read(Block & block)
if (!skip_unknown)
throw Exception("Unknown field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
/// Если ключ не найден, то пропускаем значение.
/// If the key is not found, skip the value.
NullSink sink;
readEscapedStringInto(sink, istr);
}
@ -139,7 +139,7 @@ bool TSKVRowInputStream::read(Block & block)
}
else
{
/// Единственное, что может идти без значения - это фрагмент tskv, который игнорируется.
/// The only thing that can go without value is `tskv` fragment that is ignored.
if (!(name_ref.size == 4 && 0 == memcmp(name_ref.data, "tskv", 4)))
throw Exception("Found field without value while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
}
@ -163,7 +163,7 @@ bool TSKVRowInputStream::read(Block & block)
}
}
/// Заполняем не встретившиеся столбцы значениями по-умолчанию.
/// Fill in the not met columns with default values.
for (size_t i = 0; i < columns; ++i)
if (!read_columns[i])
block.getByPosition(i).column.get()->insertDefault();

View File

@ -58,7 +58,7 @@ void TabSeparatedRowInputStream::readPrefix()
}
/** Проверка на распространённый случай ошибки - использование Windows перевода строки.
/** Check for a common error case - usage of Windows line feed.
*/
static void checkForCarriageReturn(ReadBuffer & istr)
{
@ -84,7 +84,7 @@ bool TabSeparatedRowInputStream::read(Block & block)
{
data_types[i].get()->deserializeTextEscaped(*block.getByPosition(i).column.get(), istr);
/// пропускаем разделители
/// skip separators
if (i + 1 == size)
{
if (!istr.eof())
@ -112,7 +112,7 @@ String TabSeparatedRowInputStream::getDiagnosticInfo()
WriteBufferFromString out(res);
Block block = sample.cloneEmpty();
/// Вывести подробную диагностику возможно лишь если последняя и предпоследняя строка ещё находятся в буфере для чтения.
/// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer.
size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
@ -130,7 +130,7 @@ String TabSeparatedRowInputStream::getDiagnosticInfo()
if (sample.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = sample.safeGetByPosition(i).type->getName().size();
/// Откатываем курсор для чтения на начало предыдущей или текущей строки и парсим всё заново. Но теперь выводим подробную информацию.
/// Roll back the cursor to the beginning of the previous or current line and pars all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
@ -194,7 +194,7 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
if (data_types[i]->isNumeric())
{
/// Пустая строка вместо числа.
/// An empty string instead of a number.
if (curr_position == prev_position)
{
out << "ERROR: text ";
@ -237,7 +237,7 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
}
}
/// Разделители
/// Delimiters
if (i + 1 == size)
{
if (!istr.eof())

View File

@ -54,8 +54,8 @@ const Block & TotalsHavingBlockInputStream::getTotals()
{
if (!totals)
{
/** Если totals_mode == AFTER_HAVING_AUTO, нужно решить, добавлять ли в TOTALS агрегаты для строк,
* не прошедших max_rows_to_group_by.
/** If totals_mode == AFTER_HAVING_AUTO, you need to decide whether to add aggregates to TOTALS for strings,
* not passed max_rows_to_group_by.
*/
if (overflow_aggregates)
{
@ -86,7 +86,7 @@ Block TotalsHavingBlockInputStream::readImpl()
{
block = children[0]->read();
/// Блок со значениями, не вошедшими в max_rows_to_group_by. Отложим его.
/// Block with values not included in `max_rows_to_group_by`. We'll postpone it.
if (overflow_row && block && block.info.is_overflows)
{
overflow_aggregates = block;
@ -107,7 +107,7 @@ Block TotalsHavingBlockInputStream::readImpl()
}
else
{
/// Вычисляем выражение в HAVING.
/// Compute the expression in HAVING.
expression->execute(finalized);
size_t filter_column_pos = finalized.getPositionByName(filter_column_name);
@ -125,13 +125,13 @@ Block TotalsHavingBlockInputStream::readImpl()
IColumn::Filter & filter = filter_column->getData();
/// Прибавляем значения в totals (если это не было сделано ранее).
/// Add values to `totals` (if it was not already done).
if (totals_mode == TotalsMode::BEFORE_HAVING)
addToTotals(current_totals, block, nullptr);
else
addToTotals(current_totals, block, &filter);
/// Фильтруем блок по выражению в HAVING.
/// Filter the block by expression in HAVING.
size_t columns = finalized.columns();
for (size_t i = 0; i < columns; ++i)

View File

@ -41,9 +41,9 @@ bool ValuesRowInputStream::read(Block & block)
if (istr.eof() || *istr.position() == ';')
return false;
/** Как правило, это обычный формат для потокового парсинга.
* Но в качестве исключения, поддерживается также обработка произвольных выражений вместо значений.
* Это очень неэффективно. Но если выражений нет, то оверхед отсутствует.
/** Typically, this is the usual format for streaming parsing.
* But as an exception, it also supports processing arbitrary expressions instead of values.
* This is very inefficient. But if there are no expressions, then there is no overhead.
*/
ParserExpressionWithOptionalAlias parser(false);
@ -75,9 +75,9 @@ bool ValuesRowInputStream::read(Block & block)
if (!interpret_expressions)
throw;
/** Обычный потоковый парсер не смог распарсить значение.
* Попробуем распарсить его SQL-парсером как константное выражение.
* Это исключительный случай.
/** The normal streaming parser could not parse the value.
* Let's try to parse it with a SQL parser as a constant expression.
* This is an exceptional case.
*/
if (e.code() == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED
|| e.code() == ErrorCodes::CANNOT_PARSE_QUOTED_STRING
@ -85,9 +85,9 @@ bool ValuesRowInputStream::read(Block & block)
|| e.code() == ErrorCodes::CANNOT_PARSE_DATETIME
|| e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT)
{
/// TODO Работоспособность, если выражение не помещается целиком до конца буфера.
/// TODO Performance if the expression does not fit entirely to the end of the buffer.
/// Если начало значения уже не лежит в буфере.
/// If the beginning of the value is no longer in the buffer.
if (istr.count() - istr.offset() != prev_istr_bytes)
throw;

View File

@ -19,8 +19,8 @@ XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample
if (!sample_.getByPosition(i).type->isNumeric())
have_non_numeric_columns = true;
/// В качестве имён элементов будем использовать имя столбца, если оно имеет допустимый вид, или "field", иначе.
/// Условие, приведённое ниже, более строгое, чем того требует стандарт XML.
/// As element names, we will use the column name if it has a valid form, or "field", otherwise.
/// The condition below is more strict than the XML standard requires.
bool is_column_name_suitable = true;
const char * begin = fields[i].name.data();
const char * end = begin + fields[i].name.size();

View File

@ -32,7 +32,7 @@ void copyData(IBlockInputStream & from, IBlockOutputStream & to, std::atomic<boo
if (isAtomicSet(is_cancelled))
return;
/// Для вывода дополнительной информации в некоторых форматах.
/// For outputting additional information in some formats.
if (IProfilingBlockInputStream * input = dynamic_cast<IProfilingBlockInputStream *>(&from))
{
if (input->getProfileInfo().hasAppliedLimit())

View File

@ -26,7 +26,7 @@ static void glue(BlockInputStreamPtr & node, IDsMap & ids_map, ForksMap & forks_
String id = node->getID();
if (ids_map.end() != ids_map.find(id) && ids_map[id].size() > 1)
{
/// Вставить "вилку" или использовать уже готовую.
/// Insert a "fork" or use an existing one.
if (forks_map.end() == forks_map.find(id))
{
forks_map[id] = std::make_shared<ForkBlockInputStreams>(node);

View File

@ -84,7 +84,7 @@ try
{"WithHash", std::make_shared<DataTypeUInt8>()},
};
/// создаём описание, как читать данные из tab separated дампа
/// we create a description of how to read data from the tab separated dump
Block sample;
for (const auto & name_type : names_and_types_list)
@ -96,7 +96,7 @@ try
sample.insert(std::move(elem));
}
/// читаем данные из строчного tsv файла и одновременно пишем в блочный tsv файл
/// read the data from row tsv file and simultaneously write to the block tsv file
{
ReadBufferFromIStream in_buf(std::cin);
WriteBufferFromOStream out_buf(std::cout);

View File

@ -103,11 +103,11 @@ int main(int argc, char ** argv)
std::cerr << std::endl;
std::cerr << ast->getTreeID() << std::endl;
/// создаём объект существующей таблицы хит лога
/// create an object of an existing hit log table
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list));
/// читаем из неё, применяем выражение, фильтруем, и пишем в tsv виде в консоль
/// read from it, apply the expression, filter, and write in tsv form to the console
ExpressionAnalyzer analyzer(ast, context, nullptr, names_and_types_list);
ExpressionActionsChain chain;

View File

@ -93,11 +93,11 @@ try
for (const auto & name_type : names_and_types_list)
column_names.push_back(name_type.name);
/// создаём объект существующей таблицы хит лога
/// create an object of an existing hit log table
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list));
/// читаем из неё
/// read from it
if (argc == 2 && 0 == strcmp(argv[1], "read"))
{
QueryProcessingStage::Enum stage;
@ -108,7 +108,7 @@ try
copyData(*in, out3);
}
/// читаем данные из native файла и одновременно пишем в таблицу
/// read the data from the native file and simultaneously write to the table
if (argc == 2 && 0 == strcmp(argv[1], "write"))
{
ReadBufferFromFileDescriptor in1(STDIN_FILENO);

View File

@ -105,11 +105,11 @@ try
formatAST(*ast, std::cerr);
std::cerr << std::endl;
/// создаём объект существующей таблицы хит лога
/// create an object of an existing hit log table
StoragePtr table = StorageLog::create("./", "HitLog", std::make_shared<NamesAndTypesList>(names_and_types_list));
/// читаем из неё, сортируем, и пишем в tsv виде в консоль
/// read from it, sort it, and write it in tsv form to the console
Names column_names
{

View File

@ -105,12 +105,12 @@ void DataTypeArray::serializeBinaryBulk(const IColumn & column, WriteBuffer & os
if (offset > offsets.size())
return;
/** offset - с какого массива писать.
* limit - сколько массивов максимум записать, или 0, если писать всё, что есть.
* end - до какого массива заканчивается записываемый кусок.
/** offset - from which array to write.
* limit - how many arrays should be written, or 0, if you write everything that is.
* end - up to which array the recorded piece ends.
*
* nested_offset - с какого элемента внутренностей писать.
* nested_limit - сколько элементов внутренностей писать, или 0, если писать всё, что есть.
* nested_offset - from which element of the innards to write.
* nested_limit - how many elements of the innards to write, or 0, if you write everything that is.
*/
size_t end = std::min(offset + limit, offsets.size());
@ -131,7 +131,7 @@ void DataTypeArray::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, s
ColumnArray::Offsets_t & offsets = column_array.getOffsets();
IColumn & nested_column = column_array.getData();
/// Должно быть считано согласнованное с offsets количество значений.
/// Number of values correlated with `offsets` must be read.
size_t last_offset = (offsets.empty() ? 0 : offsets.back());
if (last_offset < nested_column.size())
throw Exception("Nested column longer than last offset", ErrorCodes::LOGICAL_ERROR);
@ -354,7 +354,7 @@ void DataTypeArray::serializeTextXML(const IColumn & column, size_t row_num, Wri
void DataTypeArray::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
/// Хорошего способа сериализовать массив в CSV нет. Поэтому сериализуем его в строку, а затем полученную строку запишем в CSV.
/// There is no good way to serialize an array in CSV. Therefore, we serialize it into a string, and then write the resulting string in CSV.
String s;
{
WriteBufferFromString wb(s);
@ -390,7 +390,7 @@ ColumnPtr DataTypeArray::createColumn() const
ColumnPtr DataTypeArray::createConstColumn(size_t size, const Field & field) const
{
/// Последним аргументом нельзя отдать this.
/// `this` can not be passed as the last argument.
return std::make_shared<ColumnConstArray>(size, get<const Array &>(field), std::make_shared<DataTypeArray>(nested));
}

View File

@ -43,7 +43,7 @@ void DataTypeDate::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) co
assertChar('\'', istr);
readDateText(x, istr);
assertChar('\'', istr);
static_cast<ColumnUInt16 &>(column).getData().push_back(x); /// Важно делать это в конце - для exception safety.
static_cast<ColumnUInt16 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
}
void DataTypeDate::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, bool) const

View File

@ -43,7 +43,7 @@ void DataTypeDateTime::deserializeTextQuoted(IColumn & column, ReadBuffer & istr
assertChar('\'', istr);
readDateTimeText(x, istr);
assertChar('\'', istr);
static_cast<ColumnUInt32 &>(column).getData().push_back(x); /// Важно делать это в конце - для exception safety.
static_cast<ColumnUInt32 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
}
void DataTypeDateTime::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, bool) const

View File

@ -150,7 +150,7 @@ void DataTypeEnum<Type>::serializeTextEscaped(const IColumn & column, size_t row
template <typename Type>
void DataTypeEnum<Type>::deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const
{
/// NOTE Неплохо было бы сделать без создания временного объекта - хотя бы вынести std::string наружу.
/// NOTE It would be nice to do without creating a temporary object - at least extract std::string out.
std::string name;
readEscapedString(name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(name)));
@ -284,7 +284,7 @@ Field DataTypeEnum<Type>::castToValue(const Field & value_or_name) const
}
/// Явные инстанцирования.
/// Explicit instantiations.
template class DataTypeEnum<Int8>;
template class DataTypeEnum<Int16>;

View File

@ -83,7 +83,7 @@ template <typename T>
void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{
bool has_quote = false;
if (!istr.eof() && *istr.position() == '"') /// Понимаем число как в кавычках, так и без.
if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without.
{
has_quote = true;
++istr.position();
@ -133,7 +133,7 @@ Field DataTypeNumberBase<T>::getDefault() const
template <typename T>
void DataTypeNumberBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
/// ColumnVector<T>::value_type - более узкий тип. Например, UInt8, когда тип Field - UInt64
/// ColumnVector<T>::value_type is a narrower type. For example, UInt8, when the Field type is UInt64
typename ColumnVector<T>::value_type x = get<typename NearestFieldType<FieldType>::Type>(field);
writeBinary(x, ostr);
}

View File

@ -126,7 +126,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars_t & data, Column
if (size)
{
#if __SSE2__
/// Оптимистичная ветка, в которой возможно более эффективное копирование.
/// An optimistic branch in which more efficient copying is possible.
if (offset + 16 * UNROLL_TIMES <= data.allocated_size() && istr.position() + size + 16 * UNROLL_TIMES <= istr.buffer().end())
{
const __m128i * sse_src_pos = reinterpret_cast<const __m128i *>(istr.position());
@ -135,7 +135,7 @@ static NO_INLINE void deserializeBinarySSE2(ColumnString::Chars_t & data, Column
while (sse_src_pos < sse_src_end)
{
/// NOTE gcc 4.9.2 разворачивает цикл, но почему-то использует только один xmm регистр.
/// NOTE gcc 4.9.2 expands the loop, but for some reason uses only one xmm register.
///for (size_t j = 0; j < UNROLL_TIMES; ++j)
/// _mm_storeu_si128(sse_dst_pos + j, _mm_loadu_si128(sse_src_pos + j));
@ -177,18 +177,18 @@ void DataTypeString::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr,
if (avg_value_size_hint && avg_value_size_hint > sizeof(offsets[0]))
{
/// Выбрано наугад.
/// Randomly selected.
constexpr auto avg_value_size_hint_reserve_multiplier = 1.2;
avg_chars_size = (avg_value_size_hint - sizeof(offsets[0])) * avg_value_size_hint_reserve_multiplier;
}
else
{
/** Небольшая эвристика для оценки того, что в столбце много пустых строк.
* В этом случае, для экономии оперативки, будем говорить, что средний размер значения маленький.
/** A small heuristic to evaluate that there are a lot of empty lines in the column.
* In this case, to save RAM, we will say that the average size of the value is small.
*/
if (istr.position() + sizeof(UInt32) <= istr.buffer().end()
&& unalignedLoad<UInt32>(istr.position()) == 0) /// Первые 4 строки находятся в буфере и являются пустыми.
&& unalignedLoad<UInt32>(istr.position()) == 0) /// The first 4 rows are in the buffer and are empty.
{
avg_chars_size = 1;
}

View File

@ -62,7 +62,7 @@ void DataTypeTuple::serializeBinary(const IColumn & column, size_t row_num, Writ
template <typename F>
static void deserializeSafe(const DataTypes & elems, IColumn & column, ReadBuffer & istr, F && impl)
{
/// Используем допущение, что кортежей нулевого размера не бывает.
/// We use the assumption that tuples of zero size do not exist.
size_t old_size = extractElementColumn(column, 0).size();
try

View File

@ -69,13 +69,13 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
if (x.empty())
throw Exception("Cannot infer type of empty array", ErrorCodes::EMPTY_DATA_PASSED);
/** Тип массива нужно вывести по типу его элементов.
* Если элементы - числа, то нужно выбрать наименьший общий тип, если такой есть,
* или кинуть исключение.
* Код похож на NumberTraits::ResultOfIf, но тем кодом трудно здесь непосредственно воспользоваться.
/** The type of the array should be determined by the type of its elements.
* If the elements are numbers, then select the smallest common type, if any,
* or throw an exception.
* The code is similar to NumberTraits::ResultOfIf, but it's hard to use this code directly.
*
* Также заметим, что Float32 не выводится, вместо этого используется только Float64.
* Это сделано потому что литералов типа Float32 не бывает в запросе.
* Also notice that Float32 is not output, only Float64 is used instead.
* This is done because Float32 type literals do not exist in the query.
*/
bool has_string = false;
@ -209,7 +209,7 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
if (max_unsigned_bits >= max_signed_bits)
{
/// Беззнаковый тип не помещается в знаковый. Надо увеличить количество бит.
/// An unsigned type does not fit into a signed type. It is necessary to increase the number of bits.
if (max_bits == 8)
return wrap_into_array(std::make_shared<DataTypeInt16>());
if (max_bits == 16)
@ -221,7 +221,7 @@ DataTypePtr FieldToDataType::operator() (Array & x) const
}
else
{
/// Беззнаковый тип помещается в знаковый.
/// An unsigned type is placed in a signed type.
if (max_bits == 8)
return wrap_into_array(std::make_shared<DataTypeInt8>());
if (max_bits == 16)

View File

@ -96,14 +96,14 @@ DatabaseCloud::DatabaseCloud(
void loadTables(Context & context, ThreadPool * thread_pool, bool has_force_restore_data_flag)
{
/// Ничего не делаем - все таблицы загружаются лениво.
/// Do nothing - all tables are loaded lazily.
}
Hash DatabaseCloud::getTableHash(const String & table_name) const
{
SipHash hash;
hash.update(name.data(), name.size() + 1); /// Хэшируем также нулевой байт в качестве разделителя.
hash.update(name.data(), name.size() + 1); /// Hashing also a zero byte as a separator.
hash.update(table_name.data(), table_name.size());
Hash res;
@ -143,16 +143,16 @@ String DatabaseCloud::getTableDefinitionFromHash(Hash hash) const
}
/** Описание одной таблицы в списке таблиц в ZooKeeper.
* Без имени таблицы (правая часть отображения).
/** Description of one table in the list of tables in ZooKeeper.
* No table name (right side of map).
*/
struct TableDescription
{
/// Хэш от структуры таблицы. Сама структура хранится отдельно.
/// Hash of the table structure. The structure itself is stored separately.
Hash definition_hash;
/// Имя локальной таблицы для хранения данных. Может быть пустым, если в таблицу ещё ничего не записывали.
/// The name of the local table to store data. It can be empty if nothing else has been written to the table.
String local_table_name;
/// Список хостов, на которых расположены данные таблицы. Может быть пустым, если в таблицу ещё ничего не записывали.
/// The list of hosts on which the table data is located. It can be empty if nothing else has been written to the table.
std::vector<String> hosts;
void write(WriteBuffer & buf) const
@ -169,13 +169,13 @@ struct TableDescription
};
/** Множество таблиц в ZooKeeper.
* Точнее, его кусок, относящийся к одной ноде.
* (Всё множество разбито по TABLE_TO_NODE_DIVISOR нод.)
/** Set of tables in ZooKeeper.
* More precisely, its part, referring to one node.
* (The whole set is broken into `TABLE_TO_NODE_DIVISOR` nodes.)
*/
struct TableSet
{
/// Имя -> описание. В упорядоченном виде, данные будут лучше сжиматься.
/// Name -> description. In an ordered form, the data will be better compressed.
using Container = std::map<String, TableDescription>;
Container map;
@ -199,7 +199,7 @@ struct TableSet
{
writeCString("Version 1\n", buf);
CompressedWriteBuffer out(buf); /// NOTE Можно уменьшить размер выделяемого буфера.
CompressedWriteBuffer out(buf); /// NOTE You can reduce size of allocated buffer.
for (const auto & kv : map)
{
writeBinary(kv.first, out);
@ -223,13 +223,13 @@ struct TableSet
};
/** Множество локальных таблиц в ZooKeeper.
* Точнее, его кусок, относящийся к одной ноде.
* (Всё множество разбито по TABLE_TO_NODE_DIVISOR нод.)
/** Set of local tables in ZooKeeper.
* More precisely, its part, referring to one node.
* (The whole set is broken into `TABLE_TO_NODE_DIVISOR` nodes.)
*/
struct LocalTableSet
{
/// Хэш от имени -> хэш от структуры.
/// Hash of name -> hash of structure.
using Container = std::map<Hash, Hash>;
Container map;
@ -253,7 +253,7 @@ struct LocalTableSet
{
writeCString("Version 1\n", buf);
CompressedWriteBuffer out(buf); /// NOTE Можно уменьшить размер выделяемого буфера.
CompressedWriteBuffer out(buf); /// NOTE You can reduce size of allocated buffer.
for (const auto & kv : map)
{
writePODBinary(kv.first, out);
@ -277,9 +277,9 @@ struct LocalTableSet
};
/** Модифицировать TableSet или LocalTableSet, сериализованный в ZooKeeper, как единое целое.
* Делается compare-and-swap. Функция transform может вызываться много раз.
* Если transform возвращает false, то считается, что модифицировать нечего и результат не сохраняется в ZK.
/** Modify TableSet or LocalTableSet, serialized in ZooKeeper, as a single unit.
* The compare-and-swap is done. `transform` function can be called many times.
* If `transform` returns false, it is considered that there is nothing to modify and the result is not saved in ZK.
*/
template <typename TableSet, typename F>
static void modifyTableSet(zkutil::ZooKeeperPtr & zookeeper, const String & path, F && transform)
@ -300,7 +300,7 @@ static void modifyTableSet(zkutil::ZooKeeperPtr & zookeeper, const String & path
if (code == ZOK)
break;
else if (code == ZBADVERSION)
continue; /// Узел успели поменять - попробуем ещё раз.
continue; /// Node was changed meanwhile - we'll try again.
else
throw zkutil::KeeperException(code, path);
}
@ -343,7 +343,7 @@ static void modifyTwoTableSets(zkutil::ZooKeeperPtr & zookeeper, const String &
if (code == ZOK)
break;
else if (code == ZBADVERSION)
continue; /// Узел успели поменять - попробуем ещё раз.
continue; /// Node was changed meanwhile - we'll try again.
else
throw zkutil::KeeperException(code, path1 + ", " + path2);
}
@ -352,8 +352,8 @@ static void modifyTwoTableSets(zkutil::ZooKeeperPtr & zookeeper, const String &
bool DatabaseCloud::isTableExist(const String & table_name) const
{
/// Ищем локальную таблицу в кэше локальных таблиц или в файловой системе в path.
/// Если не нашли - ищем облачную таблицу в ZooKeeper.
/// We are looking for a local table in the local table cache or in the file system in `path`.
/// If you do not find it, look for the cloud table in ZooKeeper.
{
std::lock_guard<std::mutex> lock(local_tables_mutex);
@ -378,8 +378,8 @@ bool DatabaseCloud::isTableExist(const String & table_name) const
StoragePtr DatabaseCloud::tryGetTable(const String & table_name)
{
/// Ищем локальную таблицу.
/// Если не нашли - ищем облачную таблицу в ZooKeeper.
/// We are looking for a local table.
/// If you do not find it, look for the cloud table in ZooKeeper.
{
std::lock_guard<std::mutex> lock(local_tables_mutex);
@ -399,11 +399,11 @@ StoragePtr DatabaseCloud::tryGetTable(const String & table_name)
Hash table_hash = getTableHash(table_name);
String definition = getTableDefinitionFromHash(local_tables_info.map.at(table_hash));
/// Инициализируем локальную таблицу.
/// Initialize local table.
{
std::lock_guard<std::mutex> lock(local_tables_mutex);
/// А если таблицу только что создали?
/// And if the table has just been created?
auto it = local_tables_cache.find(table_name);
if (it != local_tables_cache.end())
return it->second;
@ -426,13 +426,13 @@ StoragePtr DatabaseCloud::tryGetTable(const String & table_name)
const TableDescription & description = tables_info.at(table_name);
String definition = getTableDefinitionFromHash(description.definition_hash);
/// TODO Инициализация объекта StorageCloud
/// TODO Initialization of `StorageCloud` object
return {};
}
}
/// Список таблиц может быть неконсистентным, так как он получается неатомарно, а по кускам, по мере итерации.
/// The list of tables can be inconsistent, as it is obtained not atomically, but in pieces, while iterating.
class DatabaseCloudIterator : public IDatabaseIterator
{
private:
@ -467,7 +467,7 @@ private:
table_set = TableSet(zookeeper->get(zookeeper_path + "/" + *nodes_iterator));
table_set_iterator = table_set.map.begin();
}
while (!table_set.map.empty()); /// Пропускаем пустые table set-ы.
while (!table_set.map.empty()); /// Skip empty table sets.
return true;
}
@ -504,7 +504,7 @@ public:
{
String definition = parent().getTableDefinitionFromHash(table_set_iterator->second.definition_hash);
/// TODO Инициализация объекта StorageCloud
/// TODO Initialization of `StorageCloud` object
return {};
}
};
@ -518,7 +518,7 @@ DatabaseIteratorPtr DatabaseCloud::getIterator()
bool DatabaseCloud::empty() const
{
/// Есть хотя бы один непустой узел среди списков таблиц.
/// There is at least one non-empty node among the list of tables.
zkutil::ZooKeeperPtr zookeeper = context.getZooKeeper();
Strings nodes = zookeeper->getChildren(zookeeper_path + "/tables/" + name);
@ -597,7 +597,7 @@ void DatabaseCloud::createTable(
{
zkutil::ZooKeeperPtr zookeeper = context.getZooKeeper();
/// Добавляем в ZK информацию о структуре таблицы.
/// Add information about the table structure to ZK.
String definition = getTableDefinitionFromCreateQuery(query);
Hash definition_hash = getHashForTableDefinition(definition);
String zookeeper_definition_path = zookeeper_path + "/table_definitions/" + hashToHex(definition_hash);
@ -610,21 +610,21 @@ void DatabaseCloud::createTable(
}
else
{
/// Более редкая ветка, так как уникальных определений таблиц немного.
/// Есть race condition, при котором узел уже существует, но проверка на логическую ошибку (см. выше) не будет осуществлена.
/// Это не имеет значения.
/// Кстати, узлы в table_definitions никогда не удаляются.
/// A rarer branch, since there are few unique table definitions.
/// There is a race condition in which the node already exists, but a check for a logical error (see above) will not be performed.
/// It does not matter.
/// By the way, nodes in `table_definitions` are never deleted.
zookeeper->tryCreate(zookeeper_definition_path, definition, zkutil::CreateMode::Persistent);
}
if (engine != "Cloud")
{
/// Если локальная таблица.
/// If the local table.
String table_name_escaped = escapeForFileName(table_name);
Poco::File(data_path + table_name_escaped).createDirectory();
Hash table_hash = getTableHash(table_name);
/// Добавляем информация о локальной таблице в ZK.
/// Add information about the local table to ZK.
modifyTableSet<LocalTableSet>(
zookeeper,
zookeeper_path + "/local_tables/" + name + "/" + getNameOfNodeWithTables(table_name),
@ -635,7 +635,7 @@ void DatabaseCloud::createTable(
return true;
});
/// Добавляем локальную таблицу в кэш.
/// Add the local table to the cache.
{
std::lock_guard<std::mutex> lock(local_tables_mutex);
if (!local_tables_cache.emplace(table_name, table).second)
@ -644,8 +644,8 @@ void DatabaseCloud::createTable(
}
else
{
/// При создании пустой облачной таблицы, локальные таблицы не создаются и серверы для них не определяются.
/// Всё делается при первой записи в таблицу.
/// When creating an empty cloud table, no local tables are created and no servers are defined for them.
/// Everything is done when you first write to the table.
TableDescription description;
description.definition_hash = definition_hash;
@ -667,15 +667,15 @@ void DatabaseCloud::removeTable(const String & table_name)
{
zkutil::ZooKeeperPtr zookeeper = context.getZooKeeper();
/// Ищем локальную таблицу.
/// Если не нашли - ищем облачную таблицу в ZooKeeper.
/// Looking for a local table.
/// If you do not find it, look for the cloud table in ZooKeeper.
String table_name_escaped = escapeForFileName(table_name);
if (Poco::File(data_path + table_name_escaped).exists())
{
Hash table_hash = getTableHash(table_name);
/// Удаляем информация о локальной таблице из ZK.
/// Delete information about the local table from ZK.
modifyTableSet<LocalTableSet>(
zookeeper,
zookeeper_path + "/local_tables/" + name + "/" + getNameOfNodeWithTables(table_name),
@ -683,13 +683,13 @@ void DatabaseCloud::removeTable(const String & table_name)
{
auto it = set.map.find(table_hash);
if (it == set.map.end())
return false; /// Таблицу уже удалили.
return false; /// The table has already been deleted.
set.map.erase(it);
return true;
});
/// Удаляем локальную таблицу из кэша.
/// Delete the local table from the cache.
{
std::lock_guard<std::mutex> lock(local_tables_mutex);
local_tables_cache.erase(table_name);
@ -697,7 +697,7 @@ void DatabaseCloud::removeTable(const String & table_name)
}
else
{
/// Удаляем таблицу из ZK, а также запоминаем список серверов, на которых расположены локальные таблицы.
/// Delete the table from ZK, and also remember the list of servers on which local tables are located.
TableDescription description;
modifyTableSet<TableSet>(
@ -707,7 +707,7 @@ void DatabaseCloud::removeTable(const String & table_name)
{
auto it = set.map.find(table_name);
if (it == set.map.end())
return false; /// Таблицу уже удалили.
return false; /// The table has already been deleted.
description = it->second;
set.map.erase(it);
@ -716,7 +716,7 @@ void DatabaseCloud::removeTable(const String & table_name)
if (!description.local_table_name.empty() && !description.hosts.empty())
{
/// Удаление локальных таблиц. TODO То ли сразу здесь, то ли в отдельном фоновом потоке.
/// Deleting local tables. TODO Whether at once here, or in a separate background thread.
}
}
}
@ -725,8 +725,8 @@ void DatabaseCloud::removeTable(const String & table_name)
void DatabaseCloud::renameTable(
const Context & context, const String & table_name, IDatabase & to_database, const String & to_table_name, const Settings & settings)
{
/// Переименовывать можно только облачные таблицы.
/// Перенос между БД не поддерживается.
/// Only cloud tables can be renamed.
/// The transfer between databases is not supported.
if (&to_database != this)
throw Exception("Moving of tables in Cloud database between databases is not supported", ErrorCodes::NOT_IMPLEMENTED);
@ -765,8 +765,8 @@ time_t DatabaseCloud::getTableMetaModTime(const String & table_name)
void DatabaseCloud::shutdown()
{
/// Нельзя удерживать блокировку во время shutdown.
/// Потому что таблицы могут внутри функции shutdown работать с БД, а mutex не рекурсивный.
/// You can not hold a lock during shutdown.
/// Because inside `shutdown` function the tables can work with database, and mutex is not recursive.
Tables local_tables_snapshot;
{
std::lock_guard<std::mutex> lock(local_tables_mutex);

View File

@ -99,8 +99,8 @@ ASTPtr DatabaseMemory::getCreateQuery(const String & table_name) const
void DatabaseMemory::shutdown()
{
/// Нельзя удерживать блокировку во время shutdown.
/// Потому что таблицы могут внутри функции shutdown работать с БД, а mutex не рекурсивный.
/// You can not hold a lock during shutdown.
/// Because inside `shutdown` function tables can work with database, and mutex is not recursive.
for (auto iterator = getIterator(); iterator->isValid(); iterator->next())
iterator->table()->shutdown();

View File

@ -62,8 +62,8 @@ static void loadTable(
readStringUntilEOF(s, in);
}
/** Пустые файлы с метаданными образуются после грубого перезапуска сервера.
* Удаляем эти файлы, чтобы чуть-чуть уменьшить работу админов по запуску.
/** Empty files with metadata are generated after a rough restart of the server.
* Remove these files to slightly reduce the work of the admins on startup.
*/
if (s.empty())
{
@ -110,11 +110,11 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
if (dir_it.name().at(0) == '.')
continue;
/// Есть файлы .sql.bak - пропускаем.
/// There are .sql.bak files - skip them.
if (endsWith(dir_it.name(), ".sql.bak"))
continue;
/// Есть файлы .sql.tmp - удаляем.
/// There are files .sql.tmp - delete.
if (endsWith(dir_it.name(), ".sql.tmp"))
{
LOG_INFO(log, "Removing file " << dir_it->path());
@ -122,7 +122,7 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
continue;
}
/// Нужные файлы имеют имена вида table_name.sql
/// The required files have names like `table_name.sql`
if (endsWith(dir_it.name(), ".sql"))
file_names.push_back(dir_it.name());
else
@ -130,9 +130,9 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
ErrorCodes::INCORRECT_FILE_NAME);
}
/** Таблицы быстрее грузятся, если их грузить в сортированном (по именам) порядке.
* Иначе (для файловой системы ext4) DirectoryIterator перебирает их в некотором порядке,
* который не соответствует порядку создания таблиц и не соответствует порядку их расположения на диске.
/** Tables load faster if they are loaded in sorted (by name) order.
* Otherwise (for the ext4 file system), `DirectoryIterator` iterates through them in some order,
* which does not correspond to order tables creation and does not correspond to order of their location on disk.
*/
std::sort(file_names.begin(), file_names.end());
@ -150,7 +150,7 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
{
const String & table = *it;
/// Сообщения, чтобы было не скучно ждать, когда сервер долго загружается.
/// Messages, so that it's not boring to wait for the server to load for a long time.
if ((++tables_processed) % PRINT_MESSAGE_EACH_N_TABLES == 0
|| watch.lockTestAndRestart(PRINT_MESSAGE_EACH_N_SECONDS))
{
@ -162,8 +162,8 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
}
};
/** packaged_task используются, чтобы исключения автоматически прокидывались в основной поток.
* Недостаток - исключения попадают в основной поток только после окончания работы всех task-ов.
/** `packaged_task` is used so that exceptions are automatically passed to the main thread.
* Disadvantage - exceptions fall into the main thread only after the end of all tasks.
*/
const size_t bunch_size = TABLES_PARALLEL_LOAD_BUNCH_SIZE;
@ -192,17 +192,17 @@ void DatabaseOrdinary::loadTables(Context & context, ThreadPool * thread_pool, b
void DatabaseOrdinary::createTable(
const String & table_name, const StoragePtr & table, const ASTPtr & query, const String & engine, const Settings & settings)
{
/// Создаём файл с метаданными, если нужно - если запрос не ATTACH.
/// В него записывается запрос на ATTACH таблицы.
/// Create a file with metadata if necessary - if the query is not ATTACH.
/// Write the query of `ATTACH table` to it.
/** Код исходит из допущения, что во всех потоках виден один и тот же порядок действий:
* - создание файла .sql.tmp;
* - добавление таблицы в tables;
* - переименование .sql.tmp в .sql.
/** The code is based on the assumption that all threads share the same order of operations
* - creating the .sql.tmp file;
* - adding a table to `tables`;
* - rename .sql.tmp to .sql.
*/
/// Был бы возможен race condition, если таблицу с одним именем одновременно создают с помощью CREATE и с помощью ATTACH.
/// Но от него есть защита - см. использование DDLGuard в InterpreterCreateQuery.
/// A race condition would be possible if a table with the same name is simultaneously created using CREATE and using ATTACH.
/// But there is protection from it - see using DDLGuard in InterpreterCreateQuery.
{
std::lock_guard<std::mutex> lock(mutex);
@ -293,7 +293,7 @@ void DatabaseOrdinary::renameTable(
if (!table)
throw Exception("Table " + name + "." + table_name + " doesn't exist.", ErrorCodes::TABLE_ALREADY_EXISTS);
/// Уведомляем таблицу о том, что она переименовывается. Если таблица не поддерживает переименование - кинется исключение.
/// Notify the table that it is renamed. If the table does not support renaming, exception is thrown.
try
{
table->rename(context.getPath() + "/data/" + escapeForFileName(to_database_concrete->name) + "/",
@ -302,7 +302,7 @@ void DatabaseOrdinary::renameTable(
}
catch (const Poco::Exception & e)
{
/// Более хорошая диагностика.
/// More good diagnostics.
throw Exception{e};
}
@ -310,7 +310,7 @@ void DatabaseOrdinary::renameTable(
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
ast_create_query.table = to_table_name;
/// NOTE Неатомарно.
/// NOTE Non-atomic.
to_database_concrete->createTable(to_table_name, table, ast, table->getName(), settings);
removeTable(table_name);
}
@ -346,8 +346,8 @@ ASTPtr DatabaseOrdinary::getCreateQuery(const String & table_name) const
void DatabaseOrdinary::shutdown()
{
/// Нельзя удерживать блокировку во время shutdown.
/// Потому что таблицы могут внутри функции shutdown работать с БД, а mutex не рекурсивный.
/// You can not hold a lock during shutdown.
/// Because inside `shutdown` function the tables can work with database, and mutex is not recursive.
for (auto iterator = getIterator(); iterator->isValid(); iterator->next())
iterator->table()->shutdown();
@ -359,7 +359,7 @@ void DatabaseOrdinary::shutdown()
void DatabaseOrdinary::drop()
{
/// Дополнительных действий по удалению не требуется.
/// No additional removal actions are required.
}
@ -372,7 +372,7 @@ void DatabaseOrdinary::alterTable(
const ColumnDefaults & column_defaults,
const ASTModifier & engine_modifier)
{
/// Считываем определение таблицы и заменяем в нём нужные части на новые.
/// Read the definition of the table and replace the necessary parts with new ones.
String table_name_escaped = escapeForFileName(name);
String table_metadata_tmp_path = path + "/" + table_name_escaped + ".sql.tmp";
@ -413,7 +413,7 @@ void DatabaseOrdinary::alterTable(
try
{
/// rename атомарно заменяет старый файл новым.
/// rename atomically replaces the old file with the new one.
Poco::File(table_metadata_tmp_path).renameTo(table_metadata_path);
}
catch (...)

View File

@ -16,7 +16,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query)
ASTPtr query_clone = query->clone();
ASTCreateQuery & create = typeid_cast<ASTCreateQuery &>(*query_clone.get());
/// Удаляем из запроса всё, что не нужно для ATTACH.
/// We remove everything that is not needed for ATTACH from the query.
create.attach = true;
create.database.clear();
create.as_database.clear();
@ -26,7 +26,7 @@ String getTableDefinitionFromCreateQuery(const ASTPtr & query)
String engine = typeid_cast<ASTFunction &>(*create.storage).name;
/// Для engine VIEW необходимо сохранить сам селект запрос, для остальных - наоборот
/// For engine VIEW it is necessary to save the SELECT query itself, for the rest - on the contrary
if (engine != "View" && engine != "MaterializedView")
create.select = nullptr;
@ -52,9 +52,9 @@ std::pair<String, StoragePtr> createTableFromDefinition(
ast_create_query.attach = true;
ast_create_query.database = database_name;
/// Не используем напрямую InterpreterCreateQuery::execute, так как:
/// - база данных ещё не создана;
/// - код проще, так как запрос уже приведён к подходящему виду.
/// We do not directly use `InterpreterCreateQuery::execute`, because
/// - the database has not been created yet;
/// - the code is simpler, since the query is already brought to a suitable form.
InterpreterCreateQuery::ColumnsInfo columns_info = InterpreterCreateQuery::getColumnsInfo(ast_create_query.columns, context);

View File

@ -19,9 +19,9 @@ bool isAttributeTypeConvertibleTo(AttributeUnderlyingType from, AttributeUnderly
if (from == to)
return true;
/** Это перечисление может быть несколько неполным и смысл может не совпадать с NumberTraits.h.
* (например тем, что целые числа нельзя преобразовать во float-ы)
* Это нормально для ограниченной области применения.
/** This enum can be somewhat incomplete and the meaning may not coincide with NumberTraits.h.
* (for example, because integers can not be converted to floats)
* This is normal for a limited usage scope.
*/
if ( (from == AttributeUnderlyingType::UInt8 && to == AttributeUnderlyingType::UInt16)
|| (from == AttributeUnderlyingType::UInt8 && to == AttributeUnderlyingType::UInt32)

View File

@ -69,8 +69,8 @@ void RegionsHierarchy::reload()
DB::assertChar('\t', in);
DB::readIntText(read_type, in);
/** Далее может быть перевод строки (старый вариант)
* или таб, население региона, перевод строки (новый вариант).
/** Then there can be a newline (old version)
* or tab, the region's population, line feed (new version).
*/
RegionPopulation population = 0;
if (!in.eof() && *in.position() == '\t')
@ -126,7 +126,7 @@ void RegionsHierarchy::reload()
new_depths .resize(max_region_id + 1);
types .resize(max_region_id + 1);
/// пропишем города и страны для регионов
/// prescribe the cities and countries for the regions
for (RegionID i = 0; i <= max_region_id; ++i)
{
if (types[i] == REGION_TYPE_CITY)

View File

@ -61,7 +61,7 @@ void RegionsNames::reload(const std::string & directory)
Chars new_chars;
StringRefs new_names_refs(initial_size, StringRef("", 0));
/// Выделим непрерывный кусок памяти, которого хватит для хранения всех имён.
/// Allocate a continuous slice of memory, which is enough to store all names.
new_chars.reserve(Poco::File(path).getSize());
while (!in.eof())

View File

@ -52,7 +52,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadAll()
BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
/// Здесь не логгируем и не обновляем время модификации, так как запрос может быть большим, и часто задаваться.
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
const auto query = query_builder.composeLoadIdsQuery(ids);
return std::make_shared<MySQLBlockInputStream>(pool.Get(), query, sample_block, max_block_size);
@ -61,7 +61,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector<UInt64> & i
BlockInputStreamPtr MySQLDictionarySource::loadKeys(
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
{
/// Здесь не логгируем и не обновляем время модификации, так как запрос может быть большим, и часто задаваться.
/// We do not log in here and do not update the modification time, as the request can be large, and often called.
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
return std::make_shared<MySQLBlockInputStream>(pool.Get(), query, sample_block, max_block_size);

View File

@ -11,8 +11,8 @@ namespace ErrorCodes
}
/** Эти функции определены в отдельных translation unit-ах.
* Это сделано для того, чтобы уменьшить потребление оперативки при сборке, и ускорить параллельную сборку.
/** These functions are defined in a separate translation units.
* This is done in order to reduce the consumption of RAM during build, and to speed up the parallel build.
*/
void registerFunctionsArithmetic(FunctionFactory &);
void registerFunctionsArray(FunctionFactory &);

View File

@ -144,7 +144,7 @@ bool tryAddField(DataTypePtr type_res, const Field & f, Array & arr)
bool FunctionArray::addField(DataTypePtr type_res, const Field & f, Array & arr) const
{
/// Иначе необходимо
/// Otherwise, it is necessary
if ( tryAddField<DataTypeUInt8, DataTypeUInt64>(type_res, f, arr)
|| tryAddField<DataTypeUInt16, DataTypeUInt64>(type_res, f, arr)
|| tryAddField<DataTypeUInt32, DataTypeUInt64>(type_res, f, arr)
@ -251,14 +251,14 @@ void FunctionArray::executeImpl(Block & block, const ColumnNumbers & arguments,
if (DataTypeTraits::removeNullable(elem.type)->getName() == observed_type->getName())
{
/// Если элемент такого же типа как результат, просто добавляем его в ответ
/// If an element of the same type as the result, just add it in response
arr.push_back((*elem.column)[0]);
}
else if (elem.type->isNull())
arr.emplace_back();
else
{
/// Иначе необходимо привести его к типу результата
/// Otherwise, you need to cast it to the result type
addField(observed_type, (*elem.column)[0], arr);
}
}
@ -553,7 +553,7 @@ struct ArrayElementStringImpl
}
else
{
/// Вставим пустую строку.
/// Insert an empty row.
result_data.resize(current_result_offset + 1);
result_data[current_result_offset] = 0;
current_result_offset += 1;
@ -1007,20 +1007,20 @@ bool FunctionArrayElement::executeTuple(Block & block, const ColumnNumbers & arg
Block & tuple_block = col_nested->getData();
size_t tuple_size = tuple_block.columns();
/** Будем вычислять функцию для кортежа внутренностей массива.
* Для этого создадим временный блок.
* Он будет состоять из следующих столбцов:
* - индекс массива, который нужно взять;
* - массив из первых элементов кортежей;
* - результат взятия элементов по индексу для массива из первых элементов кортежей;
* - массив из вторых элементов кортежей;
* - результат взятия элементов по индексу для массива из вторых элементов кортежей;
/** We will calculate the function for the tuple of the internals of the array.
* To do this, create a temporary block.
* It will consist of the following columns
* - the index of the array to be taken;
* - an array of the first elements of the tuples;
* - the result of taking the elements by the index for an array of the first elements of the tuples;
* - array of the second elements of the tuples;
* - result of taking elements by index for an array of second elements of tuples;
* ...
*/
Block block_of_temporary_results;
block_of_temporary_results.insert(block.safeGetByPosition(arguments[1]));
/// результаты взятия элементов по индексу для массивов из каждых элементов кортежей;
/// results of taking elements by index for arrays from each element of the tuples;
Block result_tuple_block;
for (size_t i = 0; i < tuple_size; ++i)
@ -2168,7 +2168,7 @@ bool FunctionEmptyArrayToSingle::executeString(
}
else
{
res_data.push_back(0); /// Пустая строка, включая ноль на конце.
res_data.push_back(0); /// An empty string, including zero at the end.
++res_string_prev_offset;
res_string_offsets.push_back(res_string_prev_offset);
@ -2641,7 +2641,7 @@ void FunctionArrayReduce::getReturnTypeAndPrerequisitesImpl(
DataTypePtr & out_return_type,
std::vector<ExpressionAction> & out_prerequisites)
{
/// Первый аргумент - константная строка с именем агрегатной функции (возможно, с параметрами в скобках, например: "quantile(0.99)").
/// The first argument is a constant string with the name of the aggregate function (possibly with parameters in parentheses, for example: "quantile(0.99)").
if (arguments.size() < 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
@ -2737,7 +2737,7 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum
size_t rows = block.rows();
/// Агрегатные функции не поддерживают константные столбцы. Поэтому, материализуем их.
/// Aggregate functions do not support constant columns. Therefore, we materialize them.
std::vector<ColumnPtr> materialized_columns;
std::vector<const IColumn *> aggregate_arguments_vec(arguments.size() - 1);

View File

@ -183,7 +183,7 @@ DataTypePtr FunctionMultiIf::getReturnTypeInternal(const DataTypes & args) const
return Conditional::getReturnTypeForArithmeticArgs(args);
else if (Conditional::hasArrayBranches(args))
{
/// NOTE Сообщения об ошибках будут относится к типам элементов массивов, что немного некорректно.
/// NOTE Error messages will refer to the types of array elements, which is slightly incorrect.
DataTypes new_args;
new_args.reserve(args.size());

View File

@ -29,38 +29,38 @@ namespace ErrorCodes
extern const int FUNCTION_IS_SPECIAL;
}
/** Вспомогательные функции:
/** Helper functions
*
* visibleWidth(x) - вычисляет приблизительную ширину при выводе значения в текстовом (tab-separated) виде на консоль.
* visibleWidth(x) - calculates the approximate width when outputting the value in a text (tab-separated) form to the console.
*
* toTypeName(x) - получить имя типа
* blockSize() - получить размер блока
* materialize(x) - материализовать константу
* ignore(...) - функция, принимающая любые аргументы, и всегда возвращающая 0.
* sleep(seconds) - спит указанное количество секунд каждый блок.
* toTypeName(x) - get the type name
* blockSize() - get the block size
* materialize(x) - materialize the constant
* ignore(...) is a function that takes any arguments, and always returns 0.
* sleep(seconds) - the specified number of seconds sleeps each block.
*
* in(x, set) - функция для вычисления оператора IN
* notIn(x, set) - и NOT IN.
* in(x, set) - function for evaluating the IN
* notIn(x, set) - and NOT IN.
*
* tuple(x, y, ...) - функция, позволяющая сгруппировать несколько столбцов
* tupleElement(tuple, n) - функция, позволяющая достать столбец из tuple.
* tuple(x, y, ...) is a function that allows you to group several columns
* tupleElement(tuple, n) is a function that allows you to retrieve a column from tuple.
*
* arrayJoin(arr) - особая функция - выполнить её напрямую нельзя;
* используется только чтобы получить тип результата соответствующего выражения.
* arrayJoin(arr) - a special function - it can not be executed directly;
* is used only to get the result type of the corresponding expression.
*
* replicate(x, arr) - создаёт массив такого же размера как arr, все элементы которого равны x;
* например: replicate(1, ['a', 'b', 'c']) = [1, 1, 1].
* replicate(x, arr) - creates an array of the same size as arr, all elements of which are equal to x;
* for example: replicate(1, ['a', 'b', 'c']) = [1, 1, 1].
*
* sleep(n) - спит n секунд каждый блок.
* sleep(n) - sleeps n seconds for each block.
*
* bar(x, min, max, width) - рисует полосу из количества символов, пропорционального (x - min) и равного width при x == max.
* bar(x, min, max, width) - draws a strip from the number of characters proportional to (x - min) and equal to width for x == max.
*
* version() - возвращает текущую версию сервера в строке.
* version() - returns the current version of the server on the line.
*
* finalizeAggregation(agg_state) - по состоянию агрегации получить результат.
* finalizeAggregation(agg_state) - get the result from the aggregation state.
*
* runningAccumulate(agg_state) - принимает состояния агрегатной функции и возвращает столбец со значениями,
* являющимися результатом накопления этих состояний для множества строк блока, от первой до текущей строки.
* runningAccumulate(agg_state) - takes the states of the aggregate function and returns a column with values,
* are the result of the accumulation of these states for a set of block lines, from the first to the current line.
*/
@ -125,7 +125,7 @@ public:
return 0;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeString>();
@ -155,7 +155,7 @@ public:
return true;
}
/// Получить имя функции.
/// Get the name of the function.
String getName() const override
{
return name;
@ -166,13 +166,13 @@ public:
return 1;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeUInt64>();
}
/// Выполнить функцию над блоком.
/// Execute the function on the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override;
};
@ -202,13 +202,13 @@ public:
return 1;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeString>();
}
/// Выполнить функцию над блоком.
/// Execute the function on the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.safeGetByPosition(result).column
@ -264,7 +264,7 @@ public:
return std::make_shared<FunctionBlockSize>();
}
/// Получить имя функции.
/// Get the function name.
String getName() const override
{
return name;
@ -280,13 +280,13 @@ public:
return 0;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeUInt64>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t size = block.rows();
@ -304,7 +304,7 @@ public:
return std::make_shared<FunctionRowNumberInBlock>();
}
/// Получить имя функции.
/// Get the name of the function.
String getName() const override
{
return name;
@ -320,13 +320,13 @@ public:
return false;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeUInt64>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t size = block.rows();
@ -341,7 +341,7 @@ public:
};
/** Инкрементальный номер блока среди вызовов этой функции. */
/** Incremental block number among calls of this function. */
class FunctionBlockNumber : public IFunction
{
private:
@ -354,7 +354,7 @@ public:
return std::make_shared<FunctionBlockNumber>();
}
/// Получить имя функции.
/// Get the function name.
String getName() const override
{
return name;
@ -370,13 +370,13 @@ public:
return false;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeUInt64>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t current_block_number = block_number++;
@ -398,7 +398,7 @@ public:
return std::make_shared<FunctionRowNumberInAllBlocks>();
}
/// Получить имя функции.
/// Get the name of the function.
String getName() const override
{
return name;
@ -414,13 +414,13 @@ public:
return false;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeUInt64>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
size_t rows_in_block = block.rows();
@ -446,7 +446,7 @@ public:
return std::make_shared<FunctionSleep>();
}
/// Получить имя функции.
/// Get the name of the function.
String getName() const override
{
return name;
@ -463,7 +463,7 @@ public:
return 1;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!typeid_cast<const DataTypeFloat64 *>(&*arguments[0]) && !typeid_cast<const DataTypeFloat32 *>(&*arguments[0])
@ -477,7 +477,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
IColumn * col = block.safeGetByPosition(arguments[0]).column.get();
@ -505,7 +505,7 @@ public:
else
throw Exception("The argument of function " + getName() + " must be constant.", ErrorCodes::ILLEGAL_COLUMN);
/// Не спим, если блок пустой.
/// We do not sleep if the block is empty.
if (size > 0)
usleep(static_cast<unsigned>(seconds * 1e6));
@ -524,7 +524,7 @@ public:
return std::make_shared<FunctionMaterialize>();
}
/// Получить имя функции.
/// Get the function name.
String getName() const override
{
return name;
@ -535,13 +535,13 @@ public:
return 1;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return arguments[0];
}
/// Выполнить функцию над блоком.
/// Apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const auto & src = block.safeGetByPosition(arguments[0]).column;
@ -595,13 +595,13 @@ public:
return 2;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return std::make_shared<DataTypeUInt8>();
}
/// Выполнить функцию над блоком.
/// Apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
/// Second argument must be ColumnSet.
@ -730,7 +730,7 @@ public:
out_return_type = elems[index - 1]->clone();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
const ColumnTuple * tuple_col = typeid_cast<const ColumnTuple *>(block.safeGetByPosition(arguments[0]).column.get());
@ -800,7 +800,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.safeGetByPosition(result).column = std::make_shared<ColumnConstUInt8>(block.rows(), 0);
@ -808,17 +808,17 @@ public:
};
/** Функция indexHint принимает любое количество любых аргументов и всегда возвращает единицу.
/** The `indexHint` function takes any number of any arguments and always returns one.
*
* Эта функция имеет особый смысл (см. ExpressionAnalyzer, PKCondition):
* - расположенные внутри неё выражения не вычисляются;
* - но при анализе индекса (выбора диапазонов для чтения), эта функция воспринимается так же,
* как если бы вместо её применения было бы само выражение.
* This function has a special meaning (see ExpressionAnalyzer, PKCondition)
* - the expressions inside it are not evaluated;
* - but when analyzing the index (selecting ranges for reading), this function is treated the same way,
* as if instead of using it the expression itself would be.
*
* Пример: WHERE something AND indexHint(CounterID = 34)
* - не читать и не вычислять CounterID = 34, но выбрать диапазоны, в которых выражение CounterID = 34 может быть истинным.
* Example: WHERE something AND indexHint(CounterID = 34)
* - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true.
*
* Функция может использоваться в отладочных целях, а также для (скрытых от пользователя) преобразований запроса.
* The function can be used for debugging purposes, as well as for (hidden from the user) query conversions.
*/
class FunctionIndexHint : public IFunction
{
@ -852,7 +852,7 @@ public:
return std::make_shared<DataTypeUInt8>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.safeGetByPosition(result).column = std::make_shared<ColumnConstUInt8>(block.rows(), 1);
@ -869,7 +869,7 @@ public:
return std::make_shared<FunctionIdentity>();
}
/// Получить имя функции.
/// Get the function name.
String getName() const override
{
return name;
@ -880,13 +880,13 @@ public:
return 1;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
return arguments.front()->clone();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
block.safeGetByPosition(result).column = block.safeGetByPosition(arguments.front()).column;
@ -904,7 +904,7 @@ public:
}
/// Получить имя функции.
/// Get the function name.
String getName() const override
{
return name;
@ -921,7 +921,7 @@ public:
return false;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const DataTypeArray * arr = typeid_cast<const DataTypeArray *>(&*arguments[0]);
@ -931,7 +931,7 @@ public:
return arr->getNestedType()->clone();
}
/// Выполнить функцию над блоком.
/// Apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
throw Exception("Function " + getName() + " must not be executed directly.", ErrorCodes::FUNCTION_IS_SPECIAL);
@ -1004,7 +1004,7 @@ public:
return 0;
}
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get the result type by argument type. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3 && arguments.size() != 4)
@ -1021,14 +1021,13 @@ public:
return std::make_shared<DataTypeString>();
}
/// Выполнить функцию над блоком.
/// apply function to the block.
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override
{
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// Уровень значения, при котором полоска имеет нулевую длину.
Int64 max
= extractConstant<Int64>(block, arguments, 2, "Third"); /// Уровень значения, при котором полоска имеет максимальную длину.
Int64 min = extractConstant<Int64>(block, arguments, 1, "Second"); /// The level at which the line has zero length.
Int64 max = extractConstant<Int64>(block, arguments, 2, "Third"); /// The level at which the line has the maximum length.
/// Максимальная ширина полоски в символах, по-умолчанию.
/// The maximum width of the bar in characters, by default.
Float64 max_width = arguments.size() == 4 ? extractConstant<Float64>(block, arguments, 3, "Fourth") : 80;
if (max_width < 1)
@ -1109,7 +1108,7 @@ private:
size_t current_offset = 0;
dst_offsets.resize(size);
dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// строки 0-terminated.
dst_chars.reserve(size * (UnicodeBar::getWidthInBytes(max_width) + 1)); /// lines 0-terminated.
for (size_t i = 0; i < size; ++i)
{
@ -1440,7 +1439,7 @@ public:
};
std::unique_ptr<char, decltype(deleter)> place{reinterpret_cast<char *>(malloc(agg_func.sizeOfData())), deleter};
agg_func.create(place.get()); /// Немного не exception-safe. Если здесь выкинется исключение, то зря вызовется destroy.
agg_func.create(place.get()); /// Not much exception-safe. If an exception is thrown out, destroy will be called in vain.
std::unique_ptr<Arena> arena = agg_func.allocatesMemoryInArena() ? std::make_unique<Arena>() : nullptr;

View File

@ -61,7 +61,7 @@ struct EmptyImpl
};
/** Вычисляет длину строки в байтах.
/** Calculates the length of a string in bytes.
*/
struct LengthImpl
{
@ -100,9 +100,9 @@ struct LengthImpl
};
/** Если строка представляет собой текст в кодировке UTF-8, то возвращает длину текста в кодовых точках.
* (не в символах: длина текста "ё" может быть как 1, так и 2, в зависимости от нормализации)
* Иначе - поведение не определено.
/** If the string is UTF-8 encoded text, it returns the length of the text in code points.
* (not in characters: the length of the text "ё" can be either 1 or 2, depending on the normalization)
* Otherwise, the behavior is undefined.
*/
struct LengthUTF8Impl
{
@ -227,7 +227,7 @@ private:
}
};
/** Разворачивает строку в байтах.
/** Expands the string in bytes.
*/
struct ReverseImpl
{
@ -269,9 +269,9 @@ struct ReverseImpl
};
/** Разворачивает последовательность кодовых точек в строке в кодировке UTF-8.
* Результат может не соответствовать ожидаемому, так как модифицирующие кодовые точки (например, диакритика) могут примениться не к тем символам.
* Если строка не в кодировке UTF-8, то поведение не определено.
/** Expands the sequence of code points in a UTF-8 encoded string.
* The result may not match the expected result, because modifying code points (for example, diacritics) may be applied to another symbols.
* If the string is not encoded in UTF-8, then the behavior is undefined.
*/
struct ReverseUTF8Impl
{
@ -413,14 +413,14 @@ void LowerUpperUTF8Impl<not_case_lower_bound, not_case_upper_bound, to_case, cyr
}
else if (src + 1 < src_end && src[0] == 0xC2u)
{
/// Пунктуация U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
/// Punctuation U+0080 - U+00BF, UTF-8: C2 80 - C2 BF
*dst++ = *src++;
*dst++ = *src++;
}
else if (src + 2 < src_end && src[0] == 0xE2u)
{
/// Символы U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
*dst++ = *src++;
/// Characters U+2000 - U+2FFF, UTF-8: E2 80 80 - E2 BF BF
*dst++ = *src++;
*dst++ = *src++;
*dst++ = *src++;
}
@ -511,7 +511,7 @@ void LowerUpperUTF8Impl<not_case_lower_bound, not_case_upper_bound, to_case, cyr
}
/** Выделяет подстроку в строке, как последовательности байт.
/** Selects a substring in a string, as a sequence of bytes.
*/
struct SubstringImpl
{
@ -585,8 +585,8 @@ struct SubstringImpl
};
/** Если строка в кодировке UTF-8, то выделяет в ней подстроку кодовых точек.
* Иначе - поведение не определено.
/** If the string is encoded in UTF-8, then it selects a substring of code points in it.
* Otherwise, the behavior is undefined.
*/
struct SubstringUTF8Impl
{
@ -741,7 +741,7 @@ public:
}
else if (const ColumnFixedString * col = typeid_cast<const ColumnFixedString *>(&*column))
{
/// Для фиксированной строки, только функция lengthUTF8 возвращает не константу.
/// For a fixed string only `lengthUTF8` function returns not a constant.
if ("lengthUTF8" != getName())
{
ResultType res = 0;
@ -793,7 +793,7 @@ public:
};
/// Также работает над массивами.
/// Also works with arrays.
class FunctionReverse : public IFunction
{
public:
@ -997,7 +997,7 @@ private:
const ColumnConstString * c0_const = typeid_cast<const ColumnConstString *>(c0);
const ColumnConstString * c1_const = typeid_cast<const ColumnConstString *>(c1);
/// Результат - const string
/// The result is const string
if (c0_const && c1_const)
{
auto c_res = std::make_shared<ColumnConstString>(c0_const->size(), "");

View File

@ -58,7 +58,7 @@ struct PositionCaseSensitiveASCII
struct PositionCaseInsensitiveASCII
{
/// Здесь не используется Volnitsky, потому что один человек померял, что так лучше. Будет хорошо, если вы подвергнете это сомнению.
/// `Volnitsky` is not used here, because one person has measured that this is better. It will be good if you question it.
using SearcherInBigHaystack = ASCIICaseInsensitiveStringSearcher;
using SearcherInSmallHaystack = LibCASCIICaseInsensitiveStringSearcher;
@ -115,7 +115,7 @@ struct PositionCaseSensitiveUTF8
struct PositionCaseInsensitiveUTF8
{
using SearcherInBigHaystack = VolnitskyImpl<false, false>;
using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Очень неоптимально.
using SearcherInSmallHaystack = UTF8CaseInsensitiveStringSearcher; /// TODO Very suboptimal.
static SearcherInBigHaystack createSearcherInBigHaystack(const char * needle_data, size_t needle_size, size_t haystack_size_hint)
{
@ -147,7 +147,7 @@ struct PositionImpl
{
using ResultType = UInt64;
/// Поиск одной подстроки во многих строках.
/// Find one substring in many strings.
static void vector_constant(const ColumnString::Chars_t & data,
const ColumnString::Offsets_t & offsets,
const std::string & needle,
@ -157,22 +157,22 @@ struct PositionImpl
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// Текущий индекс в массиве строк.
/// Current index in the array of strings.
size_t i = 0;
typename Impl::SearcherInBigHaystack searcher = Impl::createSearcherInBigHaystack(needle.data(), needle.size(), end - pos);
/// Искать будем следующее вхождение сразу во всех строках.
/// We will search for the next occurrence in all strings at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Определим, к какому индексу оно относится.
/// Determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = 0;
++i;
}
/// Проверяем, что вхождение не переходит через границы строк.
/// We check that the entry does not pass through the boundaries of strings.
if (pos + needle.size() < begin + offsets[i])
{
size_t prev_offset = i != 0 ? offsets[i - 1] : 0;
@ -188,7 +188,7 @@ struct PositionImpl
memset(&res[i], 0, (res.size() - i) * sizeof(res[0]));
}
/// Поиск одной подстроки в одной строке.
/// Search for substring in string.
static void constant_constant(std::string data, std::string needle, UInt64 & res)
{
Impl::toLowerIfNeed(data);
@ -201,7 +201,7 @@ struct PositionImpl
res = 1 + Impl::countChars(data.data(), data.data() + res);
}
/// Поиск каждой раз разной одной подстроки в каждой раз разной строке.
/// Search each time for a different single substring inside each time different string.
static void vector_vector(const ColumnString::Chars_t & haystack_data,
const ColumnString::Offsets_t & haystack_offsets,
const ColumnString::Chars_t & needle_data,
@ -220,17 +220,17 @@ struct PositionImpl
if (0 == needle_size)
{
/// Пустая строка всегда находится в самом начале haystack.
/// An empty string is always at the very beginning of `haystack`.
res[i] = 1;
}
else
{
/// Предполагается, что StringSearcher не очень сложно инициализировать.
/// It is assumed that the StringSearcher is not very difficult to initialize.
typename Impl::SearcherInSmallHaystack searcher
= Impl::createSearcherInSmallHaystack(reinterpret_cast<const char *>(&needle_data[prev_needle_offset]),
needle_offsets[i] - prev_needle_offset - 1); /// нулевой байт на конце
needle_offsets[i] - prev_needle_offset - 1); /// zero byte at the end
/// searcher возвращает указатель на найденную подстроку или на конец haystack.
/// searcher returns a pointer to the found substring or to the end of `haystack`.
size_t pos = searcher.search(&haystack_data[prev_haystack_offset], &haystack_data[haystack_offsets[i] - 1])
- &haystack_data[prev_haystack_offset];
@ -248,13 +248,13 @@ struct PositionImpl
}
}
/// Поиск многих подстрок в одной строке.
/// Find many substrings in one line.
static void constant_vector(const String & haystack,
const ColumnString::Chars_t & needle_data,
const ColumnString::Offsets_t & needle_offsets,
PaddedPODArray<UInt64> & res)
{
// NOTE Можно было бы использовать индексацию haystack. Но это - редкий случай.
// NOTE You could use haystack indexing. But this is a rare case.
ColumnString::Offset_t prev_needle_offset = 0;
@ -291,7 +291,7 @@ struct PositionImpl
};
/// Сводится ли выражение LIKE к поиску подстроки в строке?
/// Is the LIKE expression reduced to finding a substring in a string?
inline bool likePatternIsStrstr(const String & pattern, String & res)
{
res = "";
@ -346,30 +346,30 @@ struct MatchImpl
PaddedPODArray<UInt8> & res)
{
String strstr_pattern;
/// Простой случай, когда выражение LIKE сводится к поиску подстроки в строке
/// A simple case where the LIKE expression reduces to finding a substring in a string
if (like && likePatternIsStrstr(pattern, strstr_pattern))
{
const UInt8 * begin = &data[0];
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// Текущий индекс в массиве строк.
/// The current index in the array of strings.
size_t i = 0;
/// TODO Надо сделать так, чтобы searcher был общим на все вызовы функции.
/// TODO You need to make that `searcher` is common to all the calls of the function.
Volnitsky searcher(strstr_pattern.data(), strstr_pattern.size(), end - pos);
/// Искать будем следующее вхождение сразу во всех строках.
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Определим, к какому индексу оно относится.
/// Let's determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = revert;
++i;
}
/// Проверяем, что вхождение не переходит через границы строк.
/// We check that the entry does not pass through the boundaries of strings.
if (pos + strstr_pattern.size() < begin + offsets[i])
res[i] = !revert;
else
@ -379,7 +379,7 @@ struct MatchImpl
++i;
}
/// Хвостик, в котором не может быть подстрок.
/// Tail, in which there can be no substring.
memset(&res[i], revert, (res.size() - i) * sizeof(res[0]));
}
else
@ -390,13 +390,13 @@ struct MatchImpl
std::string required_substring;
bool is_trivial;
bool required_substring_is_prefix; /// для anchored выполнения регекспа.
bool required_substring_is_prefix; /// for `anchored` execution of the regexp.
regexp->getAnalyzeResult(required_substring, is_trivial, required_substring_is_prefix);
if (required_substring.empty())
{
if (!regexp->getRE2()) /// Пустой регексп. Всегда матчит.
if (!Regexp->getRE2()) /// An empty regexp. Always matches.
{
memset(&res[0], 1, size * sizeof(res[0]));
}
@ -420,31 +420,31 @@ struct MatchImpl
}
else
{
/// NOTE Это почти совпадает со случаем likePatternIsStrstr.
/// NOTE This almost matches with the case of LikePatternIsStrstr.
const UInt8 * begin = &data[0];
const UInt8 * pos = begin;
const UInt8 * end = pos + data.size();
/// Текущий индекс в массиве строк.
/// The current index in the array of strings.
size_t i = 0;
Volnitsky searcher(required_substring.data(), required_substring.size(), end - pos);
/// Искать будем следующее вхождение сразу во всех строках.
/// We will search for the next occurrence in all rows at once.
while (pos < end && end != (pos = searcher.search(pos, end - pos)))
{
/// Определим, к какому индексу оно относится.
/// Determine which index it refers to.
while (begin + offsets[i] <= pos)
{
res[i] = revert;
++i;
}
/// Проверяем, что вхождение не переходит через границы строк.
/// We check that the entry does not pass through the boundaries of strings.
if (pos + strstr_pattern.size() < begin + offsets[i])
{
/// И если не переходит - при необходимости, проверяем регекспом.
/// And if it does not, if necessary, we check the regexp.
if (is_trivial)
res[i] = !revert;
@ -453,9 +453,9 @@ struct MatchImpl
const char * str_data = reinterpret_cast<const char *>(&data[i != 0 ? offsets[i - 1] : 0]);
size_t str_size = (i != 0 ? offsets[i] - offsets[i - 1] : offsets[0]) - 1;
/** Даже в случае required_substring_is_prefix используем UNANCHORED проверку регекспа,
* чтобы он мог сматчиться, когда required_substring встречается в строке несколько раз,
* и на первом вхождении регексп не матчит.
/** Even in the case of `required_substring_is_prefix` use UNANCHORED check for regexp,
* so that it can match when `required_substring` occurs into the line several times,
* and at the first occurrence, the regexp is not a match.
*/
if (required_substring_is_prefix)
@ -769,21 +769,21 @@ struct ReplaceStringImpl
size_t size = offsets.size();
res_offsets.resize(size);
/// Текущий индекс в массиве строк.
/// The current index in the array of strings.
size_t i = 0;
Volnitsky searcher(needle.data(), needle.size(), end - pos);
/// Искать будем следующее вхождение сразу во всех строках.
/// We will search for the next occurrence in all rows at once.
while (pos < end)
{
const UInt8 * match = searcher.search(pos, end - pos);
/// Копируем данные без изменения
/// Copy the data without changing
res_data.resize(res_data.size() + (match - pos));
memcpy(&res_data[res_offset], pos, match - pos);
/// Определим, к какому индексу оно относится.
/// Determine which index it belongs to.
while (i < offsets.size() && begin + offsets[i] <= match)
{
res_offsets[i] = res_offset + ((begin + offsets[i]) - pos);
@ -791,14 +791,14 @@ struct ReplaceStringImpl
}
res_offset += (match - pos);
/// Если дошли до конца, пора остановиться
/// If you have reached the end, it's time to stop
if (i == offsets.size())
break;
/// Правда ли, что с этой строкой больше не надо выполнять преобразования.
/// Is it true that this line no longer needs to perform transformations.
bool can_finish_current_string = false;
/// Проверяем, что вхождение не переходит через границы строк.
/// We check that the entry does not go through the boundaries of strings.
if (match + needle.size() < begin + offsets[i])
{
res_data.resize(res_data.size() + replacement.size());
@ -842,21 +842,21 @@ struct ReplaceStringImpl
res_data.reserve(data.size());
res_offsets.resize(size);
/// Текущий индекс в массиве строк.
/// The current index in the string array.
size_t i = 0;
Volnitsky searcher(needle.data(), needle.size(), end - pos);
/// Искать будем следующее вхождение сразу во всех строках.
/// We will search for the next occurrence in all rows at once.
while (pos < end)
{
const UInt8 * match = searcher.search(pos, end - pos);
/// Копируем данные без изменения
/// Copy the data without changing
res_data.resize(res_data.size() + (match - pos));
memcpy(&res_data[res_offset], pos, match - pos);
/// Определим, к какому индексу оно относится.
/// Let's determine which index it belongs to.
while (i < size && begin + n * (i + 1) <= match)
{
res_offsets[i] = res_offset + ((begin + n * (i + 1)) - pos) + 1;
@ -864,14 +864,14 @@ struct ReplaceStringImpl
}
res_offset += (match - pos);
/// Если дошли до конца, пора остановиться
/// If you have reached the end, it's time to stop
if (i == size)
break;
/// Правда ли, что с этой строкой больше не надо выполнять преобразования.
/// Is it true that this line no longer needs to perform conversions.
bool can_finish_current_string = false;
/// Проверяем, что вхождение не переходит через границы строк.
/// We check that the entry does not pass through the boundaries of strings.
if (match + needle.size() - 1 < begin + n * (i + 1))
{
res_data.resize(res_data.size() + replacement.size());

View File

@ -43,7 +43,7 @@ using UInt8ColumnPtrs = std::vector<const ColumnUInt8 *>;
template <typename Op, size_t N>
struct AssociativeOperationImpl
{
/// Выбрасывает N последних столбцов из in (если их меньше, то все) и кладет в result их комбинацию.
/// Throws the last N columns from `in` (if there are less, then all) and puts their combination into `result`.
static void execute(UInt8ColumnPtrs & in, UInt8Container & result)
{
if (N > in.size())
@ -65,11 +65,11 @@ struct AssociativeOperationImpl
const UInt8Container & vec;
AssociativeOperationImpl<Op, N - 1> continuation;
/// Запоминает последние N столбцов из in.
/// Remembers the last N columns from in.
AssociativeOperationImpl(UInt8ColumnPtrs & in)
: vec(in[in.size() - N]->getData()), continuation(in) {}
/// Возвращает комбинацию значений в i-й строке всех столбцов, запомненных в конструкторе.
/// Returns a combination of values in the i-th row of all columns stored in the constructor.
inline UInt8 apply(size_t i) const
{
//return vec[i] ? continuation.apply(i) : 0;
@ -194,7 +194,7 @@ public:
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
/// Получить типы результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
DataTypePtr getReturnType(const DataTypes & arguments) const
{
if (arguments.size() < 2)
@ -224,11 +224,11 @@ public:
}
size_t n = in[0]->size();
/// Скомбинируем все константные столбцы в одно значение.
/// Combine all constant columns into a single value.
UInt8 const_val = 0;
bool has_consts = extractConstColumns(in, const_val);
// Если это значение однозначно определяет результат, вернем его.
// If this value uniquely determines the result, return it.
if (has_consts && (in.empty() || Impl<UInt8>::apply(const_val, 0) == Impl<UInt8>::apply(const_val, 1)))
{
if (!in.empty())
@ -238,7 +238,7 @@ public:
return;
}
/// Если это значение - нейтральный элемент, забудем про него.
/// If this value is a neutral element, let's forget about it.
if (has_consts && Impl<UInt8>::apply(const_val, 0) == 0 && Impl<UInt8>::apply(const_val, 1) == 1)
has_consts = false;
@ -256,8 +256,8 @@ public:
vec_res.resize(n);
}
/// Разделим входные столбцы на UInt8 и остальные. Первые обработаем более эффективно.
/// col_res в каждый момент будет либо находится в конце uint8_in, либо не содержаться в uint8_in.
/// Divide the input columns into UInt8 and the rest. The first will be processed more efficiently.
/// col_res at each moment will either be at the end of uint8_in, or not contained in uint8_in.
UInt8ColumnPtrs uint8_in;
ColumnPlainPtrs other_in;
for (IColumn * column : in)
@ -268,7 +268,7 @@ public:
other_in.push_back(column);
}
/// Нужен хотя бы один столбец в uint8_in, чтобы было с кем комбинировать столбцы из other_in.
/// You need at least one column in uint8_in, so that you can combine columns with other_in.
if (uint8_in.empty())
{
if (other_in.empty())
@ -279,14 +279,14 @@ public:
uint8_in.push_back(col_res.get());
}
/// Эффективно скомбинируем все столбцы правильного типа.
/// Effectively combine all the columns of the correct type.
while (uint8_in.size() > 1)
{
AssociativeOperationImpl<Impl<UInt8>, 10>::execute(uint8_in, vec_res);
uint8_in.push_back(col_res.get());
}
/// По одному добавим все столбцы неправильного типа.
/// One by one, add all the columns of the wrong type.
while (!other_in.empty())
{
executeUInt8Other(uint8_in[0]->getData(), other_in.back(), vec_res);
@ -294,7 +294,7 @@ public:
uint8_in[0] = col_res.get();
}
/// Такое возможно, если среди аргументов ровно один неконстантный, и он имеет тип UInt8.
/// This is possible if there are exactly one non-constant among the arguments, and it is of type UInt8.
if (uint8_in[0] != col_res.get())
{
vec_res.assign(uint8_in[0]->getData());

View File

@ -26,14 +26,14 @@ void CachedCompressedReadBuffer::initInput()
bool CachedCompressedReadBuffer::nextImpl()
{
/// Проверим наличие разжатого блока в кэше, захватим владение этим блоком, если он есть.
/// Let's check for the presence of a decompressed block in the cache, grab the ownership of this block, if it exists.
UInt128 key = cache->hash(path, file_pos);
owned_cell = cache->get(key);
if (!owned_cell)
{
/// Если нет - надо прочитать его из файла.
/// If not, read it from the file.
initInput();
file_in->seek(file_pos);
@ -48,7 +48,7 @@ bool CachedCompressedReadBuffer::nextImpl()
owned_cell->data.resize(size_decompressed);
decompress(owned_cell->data.m_data, size_decompressed, size_compressed_without_checksum);
/// Положим данные в кэш.
/// Put data into cache.
cache->set(key, owned_cell);
}
}

View File

@ -24,11 +24,11 @@ size_t CompressedReadBuffer::readBig(char * to, size_t n)
{
size_t bytes_read = 0;
/// Если в буфере есть непрочитанные байты, то скопируем сколько надо в to.
/// If there are unread bytes in the buffer, then we copy necessary to `to`.
if (pos < working_buffer.end())
bytes_read += read(to, std::min(static_cast<size_t>(working_buffer.end() - pos), n));
/// Если надо ещё прочитать - будем, по возможности, разжимать сразу в to.
/// If you need to read more - we will, if possible, uncompress at once to `to`.
while (bytes_read < n)
{
size_t size_decompressed;
@ -37,7 +37,7 @@ size_t CompressedReadBuffer::readBig(char * to, size_t n)
if (!readCompressedData(size_decompressed, size_compressed_without_checksum))
return bytes_read;
/// Если разжатый блок помещается целиком туда, куда его надо скопировать.
/// If the decompressed block is placed entirely where it needs to be copied.
if (size_decompressed <= n - bytes_read)
{
decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum);

View File

@ -47,7 +47,7 @@ void CompressedReadBufferFromFile::seek(size_t offset_in_compressed_file, size_t
{
bytes += offset();
pos = working_buffer.begin() + offset_in_decompressed_block;
/// bytes может переполниться и получиться отрицательным, но в count() все переполнится обратно и получится правильно.
/// `bytes` can overflow and get negative, but in `count()` everything will overflow back and get right.
bytes -= offset();
}
else
@ -72,22 +72,22 @@ size_t CompressedReadBufferFromFile::readBig(char * to, size_t n)
{
size_t bytes_read = 0;
/// Если в буфере есть непрочитанные байты, то скопируем сколько надо в to.
/// If there are unread bytes in the buffer, then we copy needed to `to`.
if (pos < working_buffer.end())
bytes_read += read(to, std::min(static_cast<size_t>(working_buffer.end() - pos), n));
/// Если надо ещё прочитать - будем, по возможности, разжимать сразу в to.
/// If you need to read more - we will, if possible, decompress at once to `to`.
while (bytes_read < n)
{
size_t size_decompressed = 0;
size_t size_compressed_without_checksum = 0;
size_t new_size_compressed = readCompressedData(size_decompressed, size_compressed_without_checksum);
size_compressed = 0; /// file_in больше не указывает на конец блока в working_buffer.
size_compressed = 0; /// file_in no longer points to the end of the block in working_buffer.
if (!new_size_compressed)
return bytes_read;
/// Если разжатый блок помещается целиком туда, куда его надо скопировать.
/// If the decompressed block fits entirely where it needs to be copied.
if (size_decompressed <= n - bytes_read)
{
decompress(to + bytes_read, size_decompressed, size_compressed_without_checksum);

View File

@ -33,7 +33,7 @@ void CompressedWriteBuffer::nextImpl()
size_t compressed_size = 0;
char * compressed_buffer_ptr = nullptr;
/** Формат сжатого блока - см. CompressedStream.h
/** The format of compressed block - see CompressedStream.h
*/
switch (method)

View File

@ -5,14 +5,14 @@
namespace DB
{
/// вычисление хэша зависит от разбиения по блокам
/// поэтому нужно вычислить хэш от n полных кусочков и одного неполного
/// computation of the hash depends on the partitioning of blocks
/// so you need to compute a hash of n complete pieces and one incomplete
template <class Buffer>
void IHashingBuffer<Buffer>::calculateHash(DB::BufferBase::Position data, size_t len)
{
if (len)
{
/// если данных меньше, чем block_size то сложим их в свой буффер и посчитаем от них hash позже
/// if the data is less than `block_size`, then put them into buffer and calculate hash later
if (block_pos + len < block_size)
{
memcpy(&BufferWithOwnMemory<Buffer>::memory[block_pos], data, len);
@ -20,7 +20,7 @@ void IHashingBuffer<Buffer>::calculateHash(DB::BufferBase::Position data, size_t
}
else
{
/// если в буффер уже что-то записано, то допишем его
/// if something is already written to the buffer, then we'll add it
if (block_pos)
{
size_t n = block_size - block_pos;
@ -38,7 +38,7 @@ void IHashingBuffer<Buffer>::calculateHash(DB::BufferBase::Position data, size_t
data += block_size;
}
/// запишем остаток в свой буфер
/// write the remainder to its buffer
if (len)
{
memcpy(&BufferWithOwnMemory<Buffer>::memory[0], data, len);
@ -53,7 +53,7 @@ template class IHashingBuffer<DB::WriteBuffer>;
}
/// UInt64 это 39 символов в 10 системе счисления
/// UInt64 is 39 characters in 10th number system
static const size_t UINT64_DECIMAL_SIZE = 39;
std::string uint128ToString(uint128 data)
{

View File

@ -50,7 +50,7 @@ InterserverWriteBuffer::InterserverWriteBuffer(const std::string & host_, int po
session.setPort(port);
session.setKeepAlive(true);
/// устанавливаем таймаут
/// set the timeout
#if POCO_CLICKHOUSE_PATCH || POCO_VERSION >= 0x02000000
session.setTimeout(connection_timeout, send_timeout, receive_timeout);
#else
@ -83,7 +83,7 @@ void InterserverWriteBuffer::nextImpl()
if (!offset() || finalized)
return;
/// Для корректной работы с AsynchronousWriteBuffer, который подменяет буферы.
/// For correct work with AsynchronousWriteBuffer, which replaces buffers.
impl->set(buffer().begin(), buffer().size());
impl->position() = pos;

View File

@ -35,8 +35,8 @@ namespace ErrorCodes
}
/// Примечание: выделяется дополнительная страница, которая содежрит те данные, которые
/// не влезают в основной буфер.
/// Note: an additional page is allocated that will contain the data that
/// does not fit into the main buffer.
ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, char * existing_memory_)
: ReadBufferFromFileBase(buffer_size_ + DEFAULT_AIO_FILE_BLOCK_SIZE, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE),
fill_buffer(BufferWithOwnMemory<ReadBuffer>(internalBuffer().size(), nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)),
@ -83,8 +83,8 @@ void ReadBufferAIO::setMaxBytes(size_t max_bytes_read_)
bool ReadBufferAIO::nextImpl()
{
/// Если конец файла уже был достигнут при вызове этой функции,
/// то текущий вызов ошибочен.
/// If the end of the file has already been reached by calling this function,
/// then the current call is wrong.
if (is_eof)
return false;
@ -111,11 +111,11 @@ bool ReadBufferAIO::nextImpl()
is_started = true;
/// Если конец файла только что достигнут, больше ничего не делаем.
/// If the end of the file is just reached, do nothing else.
if (is_eof)
return true;
/// Создать асинхронный запрос.
/// Create an asynchronous request.
prepare();
request.aio_lio_opcode = IOCB_CMD_PREAD;
@ -124,7 +124,7 @@ bool ReadBufferAIO::nextImpl()
request.aio_nbytes = region_aligned_size;
request.aio_offset = region_aligned_begin;
/// Отправить запрос.
/// Send the request.
try
{
future_bytes_read = AIOContextPool::instance().post(request);
@ -168,16 +168,16 @@ off_t ReadBufferAIO::doSeek(off_t off, int whence)
off_t first_read_pos_in_file = first_unread_pos_in_file - static_cast<off_t>(working_buffer.size());
if (hasPendingData() && (new_pos_in_file >= first_read_pos_in_file) && (new_pos_in_file <= first_unread_pos_in_file))
{
/// Свдинулись, но остались в пределах буфера.
/// Moved, but remained within the buffer.
pos = working_buffer.begin() + (new_pos_in_file - first_read_pos_in_file);
}
else
{
/// Сдвинулись за пределы буфера.
/// Moved past the buffer.
pos = working_buffer.end();
first_unread_pos_in_file = new_pos_in_file;
/// Не можем использовать результат текущего асинхронного запроса.
/// We can not use the result of the current asynchronous request.
skip();
}
}
@ -238,7 +238,7 @@ void ReadBufferAIO::prepare()
{
requested_byte_count = std::min(fill_buffer.internalBuffer().size() - DEFAULT_AIO_FILE_BLOCK_SIZE, max_bytes_read);
/// Регион диска, из которого хотим читать данные.
/// Region of the disk from which we want to read data.
const off_t region_begin = first_unread_pos_in_file;
if ((requested_byte_count > std::numeric_limits<off_t>::max()) ||
@ -247,7 +247,7 @@ void ReadBufferAIO::prepare()
const off_t region_end = first_unread_pos_in_file + requested_byte_count;
/// Выровненный регион диска, из которого будем читать данные.
/// The aligned region of the disk from which we will read the data.
region_left_padding = region_begin % DEFAULT_AIO_FILE_BLOCK_SIZE;
const size_t region_right_padding = (DEFAULT_AIO_FILE_BLOCK_SIZE - (region_end % DEFAULT_AIO_FILE_BLOCK_SIZE)) % DEFAULT_AIO_FILE_BLOCK_SIZE;
@ -267,10 +267,10 @@ void ReadBufferAIO::finalize()
if ((bytes_read < 0) || (static_cast<size_t>(bytes_read) < region_left_padding))
throw Exception("Asynchronous read error on file " + filename, ErrorCodes::AIO_READ_ERROR);
/// Игнорируем излишние байты слева.
/// Ignore redundant bytes on the left.
bytes_read -= region_left_padding;
/// Игнорируем излишние байты справа.
/// Ignore redundant bytes on the right.
bytes_read = std::min(bytes_read, static_cast<off_t>(requested_byte_count));
if (bytes_read > 0)
@ -288,7 +288,7 @@ void ReadBufferAIO::finalize()
if (total_bytes_read == max_bytes_read)
is_eof = true;
/// Менять местами основной и дублирующий буферы.
/// Swap the main and duplicate buffers.
internalBuffer().swap(fill_buffer.internalBuffer());
buffer().swap(fill_buffer.buffer());
std::swap(position(), fill_buffer.position());

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