This commit is contained in:
Alexey Vasiliev 2015-05-07 11:53:24 +03:00
commit 20ede563f8
210 changed files with 4687 additions and 2390 deletions

View File

@ -8,3 +8,8 @@ else()
add_custom_target(check COMMAND ${CMAKE_CTEST_COMMAND}
--force-new-ctest-process --output-on-failure)
endif()
macro (add_check target)
add_test(test_${target} ${target})
add_dependencies(check ${target})
endmacro (add_check)

View File

@ -21,7 +21,7 @@ SELECT UserID FROM {table} WHERE UserID = 12345678901234567890;
SELECT count() FROM {table} WHERE URL LIKE '%metrika%';
SELECT SearchPhrase, any(URL), count() AS c FROM {table} WHERE URL LIKE '%metrika%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;
SELECT SearchPhrase, any(URL), any(Title), count() AS c, uniq(UserID) FROM {table} WHERE Title LIKE '%Яндекс%' AND URL NOT LIKE '%.yandex.%' AND SearchPhrase != '' GROUP BY SearchPhrase ORDER BY c DESC LIMIT 10;
SELECT * FROM {table} WHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;
SELECT * FROM {table} PREWHERE URL LIKE '%metrika%' ORDER BY EventTime LIMIT 10;
SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY EventTime LIMIT 10;
SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY SearchPhrase LIMIT 10;
SELECT SearchPhrase FROM {table} WHERE SearchPhrase != '' ORDER BY EventTime, SearchPhrase LIMIT 10;

View File

@ -13,10 +13,14 @@ namespace DB
*/
class AggregateFunctionFactory
{
public:
using FunctionNames = std::vector<std::string>;
public:
AggregateFunctionFactory();
AggregateFunctionPtr get(const String & name, const DataTypes & argument_types, int recursion_level = 0) const;
AggregateFunctionPtr tryGet(const String & name, const DataTypes & argument_types) const;
const FunctionNames & getFunctionNames() const;
bool isAggregateFunctionName(const String & name, int recursion_level = 0) const;
};

View File

@ -3,10 +3,13 @@
#include <boost/program_options.hpp>
#include <boost/algorithm/string.hpp>
#include <DB/DataStreams/AsynchronousBlockInputStream.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/Interpreters/Context.h>
#include <DB/IO/copyData.h>
#include <DB/IO/ReadBufferFromIStream.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/Storages/StorageMemory.h>
#include <DB/Client/Connection.h>
#include <Poco/Net/HTMLForm.h>
#include <Poco/Net/PartHandler.h>
#include <Poco/Net/MessageHeader.h>

View File

@ -14,6 +14,13 @@
M(FileOpen) \
M(Seek) \
M(ReadBufferFromFileDescriptorRead) \
M(ReadBufferFromFileDescriptorReadBytes) \
M(WriteBufferFromFileDescriptorWrite) \
M(WriteBufferFromFileDescriptorWriteBytes) \
M(ReadBufferAIORead) \
M(ReadBufferAIOReadBytes) \
M(WriteBufferAIOWrite) \
M(WriteBufferAIOWriteBytes) \
M(ReadCompressedBytes) \
M(CompressedReadBufferBlocks) \
M(CompressedReadBufferBytes) \
@ -27,6 +34,10 @@
M(FunctionExecute) \
M(MarkCacheHits) \
M(MarkCacheMisses) \
M(CreatedReadBufferOrdinary) \
M(CreatedReadBufferAIO) \
M(CreatedWriteBufferOrdinary) \
M(CreatedWriteBufferAIO) \
\
M(ReplicatedPartFetches) \
M(ReplicatedPartFailedFetches) \

View File

@ -17,6 +17,9 @@
/// Насколько секунд можно максимально задерживать вставку в таблицу типа MergeTree, если в ней много недомердженных кусков.
#define DBMS_MAX_DELAY_OF_INSERT 200.0
/// Размер буфера ввода-вывода по-умолчанию.
#define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL
/// При записи данных, для сжатия выделяется буфер размером max_compress_block_size. При переполнении буфера или если в буфер
/// записано данных больше или равно, чем min_compress_block_size, то при очередной засечке, данные так же будут сжиматься
/// В результате, для маленьких столбцов (числа 1-8 байт), при index_granularity = 8192, размер блока будет 64 KБ.

View File

@ -24,23 +24,25 @@ public:
BlockOutputStreamPtr output_,
NamesAndTypesListPtr required_columns_,
const ColumnDefaults & column_defaults_,
const Context & context_)
const Context & context_,
bool only_explicit_column_defaults_)
: output(output_), required_columns(required_columns_),
column_defaults(column_defaults_), context(context_)
column_defaults(column_defaults_), context(context_),
only_explicit_column_defaults(only_explicit_column_defaults_)
{
}
AddingDefaultBlockOutputStream(BlockOutputStreamPtr output_, NamesAndTypesListPtr required_columns_, const Context & context_)
: AddingDefaultBlockOutputStream{output_, required_columns_, ColumnDefaults{}, context_}
{
}
void write(const Block & block) override
{
Block res = block;
/// Вычисляет явно указанные (в column_defaults) значения по-умолчанию.
evaluateMissingDefaults(res, *required_columns, column_defaults, context);
res.addDefaults(*required_columns);
/// Добавляет не указанные значения по-умолчанию.
if (!only_explicit_column_defaults)
res.addDefaults(*required_columns);
output->write(res);
}
@ -54,6 +56,7 @@ private:
NamesAndTypesListPtr required_columns;
const ColumnDefaults & column_defaults;
Context context;
bool only_explicit_column_defaults;
};

View File

@ -55,7 +55,10 @@ protected:
if (res)
break;
else
{
(*current_stream)->readSuffix();
++current_stream;
}
}
return res;

View File

@ -49,6 +49,9 @@ public:
return res.str();
}
/// Берёт totals только из основного источника, а не из источников подзапросов.
const Block & getTotals() override;
protected:
Block readImpl() override;

View File

@ -14,7 +14,7 @@ using Poco::SharedPtr;
/** Выполняет над блоком вычисление некоторого выражения.
* Выражение состоит из идентификаторов столбцов из блока, констант, обычных функций.
* Например: hits * 2 + 3, instr("yandex", url)
* Например: hits * 2 + 3, url LIKE '%yandex%'
* Выражение не меняет количество строк в потоке, и обрабатывает каждую строку независимо от других.
*/
class ExpressionBlockInputStream : public IProfilingBlockInputStream
@ -40,9 +40,7 @@ public:
if (IProfilingBlockInputStream * child = dynamic_cast<IProfilingBlockInputStream *>(&*children.back()))
{
totals = child->getTotals();
if (totals)
expression->execute(totals);
expression->executeOnTotals(totals);
}
return totals;

View File

@ -28,6 +28,8 @@ public:
return res.str();
}
static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows);
protected:
Block readImpl() override;

View File

@ -21,6 +21,8 @@ public:
void write(const Block & block) override;
void flush() override { ostr.next(); }
static void writeData(const IDataType & type, const ColumnPtr & column, WriteBuffer & ostr, size_t offset, size_t limit);
private:
WriteBuffer & ostr;
UInt64 client_revision;

View File

@ -70,6 +70,10 @@ protected:
if (!executed)
{
executed = true;
Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); };
aggregator.setCancellationHook(hook);
AggregatedDataVariantsPtr data_variants = executeAndMerge();
if (data_variants)

View File

@ -1,7 +1,7 @@
#pragma once
#include <list>
#include <stack>
#include <queue>
#include <atomic>
#include <thread>
#include <mutex>
@ -49,7 +49,7 @@ public:
: inputs(inputs_), max_threads(std::min(inputs_.size(), max_threads_)), handler(handler_)
{
for (size_t i = 0; i < inputs_.size(); ++i)
input_stack.emplace(inputs_[i], i);
available_inputs.emplace(inputs_[i], i);
}
~ParallelInputsProcessor()
@ -162,16 +162,16 @@ private:
/// Выбираем следующий источник.
{
std::lock_guard<std::mutex> lock(input_stack_mutex);
std::lock_guard<std::mutex> lock(available_inputs_mutex);
/// Если свободных источников нет, то этот поток больше не нужен. (Но другие потоки могут работать со своими источниками.)
if (input_stack.empty())
if (available_inputs.empty())
break;
input = input_stack.top();
input = available_inputs.front();
/// Убираем источник из очереди доступных источников.
input_stack.pop();
available_inputs.pop();
}
/// Основная работа.
@ -183,15 +183,15 @@ private:
/// Если этот источник ещё не иссяк, то положим полученный блок в очередь готовых.
{
std::lock_guard<std::mutex> lock(input_stack_mutex);
std::lock_guard<std::mutex> lock(available_inputs_mutex);
if (block)
{
input_stack.push(input);
available_inputs.push(input);
}
else
{
if (input_stack.empty())
if (available_inputs.empty())
break;
}
}
@ -214,15 +214,28 @@ private:
typedef std::vector<std::thread> ThreadsData;
ThreadsData threads;
/** Стек доступных источников, которые не заняты каким-либо потоком в данный момент.
* Стек вместо очереди - чтобы выполнять работу по чтению одного источника более последовательно.
* То есть, продолжать обработку источника, который недавно обрабатывался.
/** Набор доступных источников, которые не заняты каким-либо потоком в данный момент.
* Каждый поток берёт из этого набора один источник, вынимает из источника блок (в этот момент источник делает вычисления),
* и (если источник не исчерпан), кладёт назад в набор доступных источников.
*
* Возникает вопрос, что лучше использовать:
* - очередь (только что обработанный источник будет в следующий раз обработан позже остальных)
* - стек (только что обработанный источник будет обработан как можно раньше).
*
* Стек лучше очереди, когда надо выполнять работу по чтению одного источника более последовательно,
* и теоретически, это позволяет достичь более последовательных чтений с диска.
*
* Но при использовании стека, возникает проблема при распределённой обработке запроса:
* данные всё-время читаются только с части серверов, а на остальных серверах
* возникает таймаут при send-е, и обработка запроса завершается с исключением.
*
* Поэтому, используется очередь. Это можно улучшить в дальнейшем.
*/
typedef std::stack<InputData> InputStack;
InputStack input_stack;
typedef std::queue<InputData> AvailableInputs;
AvailableInputs available_inputs;
/// Для операций с input_stack.
std::mutex input_stack_mutex;
/// Для операций с available_inputs.
std::mutex available_inputs_mutex;
/// Сколько источников иссякло.
std::atomic<size_t> active_threads { 0 };

View File

@ -84,7 +84,7 @@ public:
if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed))
return;
if (hasNoQueryInProgress() || hasThrownException())
if (!isQueryPending() || hasThrownException())
return;
tryCancel("Cancelling query");
@ -97,7 +97,7 @@ public:
* все соединения, затем читаем и пропускаем оставшиеся пакеты чтобы
* эти соединения не остались висеть в рассихронизированном состоянии.
*/
if (established || isQueryInProgress())
if (established || isQueryPending())
parallel_replicas->disconnect();
}
@ -212,7 +212,7 @@ protected:
* - получили с одной реплики неизвестный пакет;
* - то больше читать ничего не нужно.
*/
if (hasNoQueryInProgress() || hasThrownException())
if (!isQueryPending() || hasThrownException())
return;
/** Если ещё прочитали не все данные, но они больше не нужны.
@ -251,16 +251,10 @@ protected:
parallel_replicas = std::make_unique<ParallelReplicas>(pool, parallel_replicas_settings, throttler);
}
/// Возвращает true, если запрос отправлен, а ещё не выполнен.
bool isQueryInProgress() const
/// Возвращает true, если запрос отправлен.
bool isQueryPending() const
{
return sent_query && !finished && !was_cancelled;
}
/// Возвращает true, если никакой запрос не отправлен или один запрос уже выполнен.
bool hasNoQueryInProgress() const
{
return !sent_query || finished;
return sent_query && !finished;
}
/// Возвращает true, если исключение было выкинуто.

View File

@ -5,8 +5,9 @@
#include <DB/Core/Row.h>
#include <DB/Core/ColumnNumbers.h>
#include <DB/DataStreams/MergingSortedBlockInputStream.h>
#include <statdaemons/ext/range.hpp>
#include <DB/Storages/MergeTree/PKCondition.h>
#include <statdaemons/ext/range.hpp>
#include <statdaemons/ext/map.hpp>
namespace DB
@ -62,13 +63,13 @@ private:
Names column_names_to_sum; /// Если задано - преобразуется в column_numbers_to_sum при инициализации.
ColumnNumbers column_numbers_to_sum;
/** Таблица может вложенные таблицы, обрабатываемые особым образом.
* Если название вложенной таблицы заканчинвается на `Map` и она содержит ровно два столбца,
/** Таблица может иметь вложенные таблицы, обрабатываемые особым образом.
* Если название вложенной таблицы заканчинвается на `Map` и она содержит не менее двух столбцов,
* удовлетворяющих следующим критериям:
* - первый столбец - числовой ((U)IntN, Date, DateTime), назовем его условно key,
* - второй столбец - арифметический ((U)IntN, Float32/64), условно value.
* Такая вложенная таблица воспринимается как отображение key => value и при слиянии
* ее строк выполняется слияние элементов двух множеств по key со сложением по value.
* - остальные столбцы - арифметические ((U)IntN, Float32/64), условно (values...).
* Такая вложенная таблица воспринимается как отображение key => (values...) и при слиянии
* ее строк выполняется слияние элементов двух множеств по key со сложением соответствующих (values...).
* Пример:
* [(1, 100)] + [(2, 150)] -> [(1, 100), (2, 150)]
* [(1, 100)] + [(1, 150)] -> [(1, 250)]
@ -76,11 +77,11 @@ private:
* [(1, 100), (2, 150)] + [(1, -100)] -> [(2, 150)]
*/
/// Хранит номера столбца-ключа и столбца-значения
/// Хранит номера столбца-ключа и столбцов-значений
struct map_description
{
std::size_t key_col_num;
std::size_t val_col_num;
std::vector<std::size_t> val_col_nums;
};
/// Найденные вложенные Map таблицы
@ -123,103 +124,131 @@ private:
bool operator() (Array & x) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); }
};
using map_merge_t = std::map<Field, Field>;
/// Performs insertion of a new value into nested Map
class MapSumVisitor : public StaticVisitor<void>
/** Для вложенных Map выполняется слияние по ключу с выбрасыванием строк вложенных массивов, в которых
* все элементы - нулевые.
*/
template<class TSortCursor>
void mergeMaps(Row & row, TSortCursor & cursor)
{
public:
map_merge_t & map;
const Field & key;
public:
MapSumVisitor(map_merge_t & map, const Field & key) : map(map), key(key) {}
void operator()(const UInt64 val) const
/// merge nested maps
for (const auto & map : maps_to_sum)
{
const auto it = map.find(key);
if (it == std::end(map))
map.emplace(key, Field{val});
else
it->second.get<UInt64>() += val;
}
const auto val_count = map.val_col_nums.size();
void operator()(const Int64 val) const
{
const auto it = map.find(key);
if (it == std::end(map))
map.emplace(key, Field{val});
else
it->second.get<Int64>() += val;
}
/// fetch key array reference from accumulator-row
auto & key_array_lhs = row[map.key_col_num].get<Array>();
/// returns a Field for pos-th item of val_index-th value
const auto val_getter_lhs = [&] (const auto val_index, const auto pos) -> decltype(auto) {
return row[map.val_col_nums[val_index]].get<Array>()[pos];
};
void operator()(const Float64 val) const
{
const auto it = map.find(key);
if (it == std::end(map))
map.emplace(key, Field{val});
else
it->second.get<Float64>() += val;
}
/// we will be sorting key positions, not the entire rows, to minimize actions
std::vector<std::size_t> key_pos_lhs(ext::range_iterator<std::size_t>{0},
ext::range_iterator<std::size_t>{key_array_lhs.size()});
std::sort(std::begin(key_pos_lhs), std::end(key_pos_lhs), [&] (const auto pos1, const auto pos2) {
return key_array_lhs[pos1] < key_array_lhs[pos2];
});
void operator() (Null) const { throw Exception("Cannot merge Nulls", ErrorCodes::LOGICAL_ERROR); }
void operator() (String) const { throw Exception("Cannot merge Strings", ErrorCodes::LOGICAL_ERROR); }
void operator() (Array) const { throw Exception("Cannot merge Arrays", ErrorCodes::LOGICAL_ERROR); }
};
/// copy key field from current row under cursor
const auto key_field_rhs = (*cursor->all_columns[map.key_col_num])[cursor->pos];
/// for each element of `map.val_col_nums` copy corresponding array under cursor into vector
const auto val_fields_rhs = ext::map<std::vector>(map.val_col_nums,
[&] (const auto col_num) -> decltype(auto) {
return (*cursor->all_columns[col_num])[cursor->pos];
});
/// fetch key array reference from row under cursor
const auto & key_array_rhs = key_field_rhs.get<Array>();
/// returns a Field for pos-th item of val_index-th value
const auto val_getter_rhs = [&] (const auto val_index, const auto pos) -> decltype(auto) {
return val_fields_rhs[val_index].get<Array>()[pos];
};
std::vector<std::size_t> key_pos_rhs(ext::range_iterator<std::size_t>{0},
ext::range_iterator<std::size_t>{key_array_rhs.size()});
std::sort(std::begin(key_pos_rhs), std::end(key_pos_rhs), [&] (const auto pos1, const auto pos2) {
return key_array_rhs[pos1] < key_array_rhs[pos2];
});
/// max size after merge estimation
const auto max_size = key_pos_lhs.size() + key_pos_rhs.size();
/// create arrays with a single element (it will be overwritten on first iteration)
Array key_array_result(1);
key_array_result.reserve(max_size);
std::vector<Array> val_arrays_result(val_count, Array(1));
for (auto & val_array_result : val_arrays_result)
val_array_result.reserve(max_size);
/// discard first element
auto discard_prev = true;
/// either insert or merge new element
const auto insert_or_sum = [&] (std::size_t & index, const std::vector<std::size_t> & key_pos,
const auto & key_array, auto && val_getter) {
const auto pos = key_pos[index++];
const auto & key = key_array[pos];
if (discard_prev)
{
discard_prev = false;
key_array_result.back() = key;
for (const auto val_index : ext::range(0, val_count))
val_arrays_result[val_index].back() = val_getter(val_index, pos);
}
else if (key_array_result.back() == key)
{
/// merge with same key
auto should_discard = true;
for (const auto val_index : ext::range(0, val_count))
if (apply_visitor(FieldVisitorSum{val_getter(val_index, pos)},
val_arrays_result[val_index].back()))
should_discard = false;
discard_prev = should_discard;
}
else
{
/// append new key
key_array_result.emplace_back(key);
for (const auto val_index : ext::range(0, val_count))
val_arrays_result[val_index].emplace_back(val_getter(val_index, pos));
}
};
std::size_t index_lhs = 0;
std::size_t index_rhs = 0;
/// perform 2-way merge
while (true)
if (index_lhs < key_pos_lhs.size() && index_rhs == key_pos_rhs.size())
insert_or_sum(index_lhs, key_pos_lhs, key_array_lhs, val_getter_lhs);
else if (index_lhs == key_pos_lhs.size() && index_rhs < key_pos_rhs.size())
insert_or_sum(index_rhs, key_pos_rhs, key_array_rhs, val_getter_rhs);
else if (index_lhs < key_pos_lhs.size() && index_rhs < key_pos_rhs.size())
if (key_array_lhs[key_pos_lhs[index_lhs]] < key_array_rhs[key_pos_rhs[index_rhs]])
insert_or_sum(index_lhs, key_pos_lhs, key_array_lhs, val_getter_lhs);
else
insert_or_sum(index_rhs, key_pos_rhs, key_array_rhs, val_getter_rhs);
else
break;
/// store results into accumulator-row
key_array_lhs = std::move(key_array_result);
for (const auto val_col_index : ext::range(0, val_count))
row[map.val_col_nums[val_col_index]].get<Array>() = std::move(val_arrays_result[val_col_index]);
}
}
/** Прибавить строчку под курсором к row.
* Для вложенных Map выполняется слияние по ключу с выбрасыванием нулевых элементов.
* Возвращает false, если результат получился нулевым.
*/
template<class TSortCursor>
bool addRow(Row & row, TSortCursor & cursor)
{
/// merge nested maps
for (const auto & map : maps_to_sum)
{
/// fetch key and val array references from accumulator-row
auto & key_array = row[map.key_col_num].get<Array>();
auto & val_array = row[map.val_col_num].get<Array>();
if (key_array.size() != val_array.size())
throw Exception{"Nested arrays have different sizes", ErrorCodes::LOGICAL_ERROR};
/// copy key and value fields from current row under cursor
const auto key_field_rhs = (*cursor->all_columns[map.key_col_num])[cursor->pos];
const auto val_field_rhs = (*cursor->all_columns[map.val_col_num])[cursor->pos];
/// fetch key and val array references from current row
const auto & key_array_rhs = key_field_rhs.get<Array>();
const auto & val_array_rhs = val_field_rhs.get<Array>();
if (key_array_rhs.size() != val_array_rhs.size())
throw Exception{"Nested arrays have different sizes", ErrorCodes::LOGICAL_ERROR};
map_merge_t result;
/// populate map from current row
for (const auto i : ext::range(0, key_array.size()))
apply_visitor(MapSumVisitor{result, key_array[i]}, val_array[i]);
/// merge current row into map
for (const auto i : ext::range(0, key_array_rhs.size()))
apply_visitor(MapSumVisitor{result, key_array_rhs[i]}, val_array_rhs[i]);
Array key_array_result;
Array val_array_result;
/// serialize result key-value pairs back into separate arrays of keys and values
for (const auto & pair : result)
{
/// we do not store the resulting value if it is zero
if (!apply_visitor(FieldVisitorAccurateEquals{}, pair.second, nearestFieldType(0)))
{
key_array_result.emplace_back(pair.first);
val_array_result.emplace_back(pair.second);
}
}
/// replace accumulator row key and value arrays with the merged ones
key_array = std::move(key_array_result);
val_array = std::move(val_array_result);
}
mergeMaps(row, cursor);
bool res = false; /// Есть ли хотя бы одно ненулевое число.

View File

@ -28,8 +28,10 @@ using Poco::SharedPtr;
class UnionBlockInputStream : public IProfilingBlockInputStream
{
public:
UnionBlockInputStream(BlockInputStreams inputs, unsigned max_threads)
: output_queue(max_threads), handler(*this), processor(inputs, max_threads, handler)
UnionBlockInputStream(BlockInputStreams inputs, size_t max_threads) :
output_queue(std::min(inputs.size(), max_threads)),
handler(*this),
processor(inputs, max_threads, handler)
{
children = inputs;
}

View File

@ -3,6 +3,8 @@
#include <DB/DataTypes/IDataType.h>
#include <DB/Columns/ColumnTuple.h>
#include <DB/Columns/ColumnConst.h>
#include <DB/DataStreams/NativeBlockInputStream.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
namespace DB
@ -115,14 +117,7 @@ public:
{
const ColumnTuple & real_column = static_cast<const ColumnTuple &>(column);
for (size_t i = 0, size = elems.size(); i < size; ++i)
{
const IColumn & nested_column = *real_column.getData().getByPosition(i).column;
if (nested_column.isConst())
elems[i]->serializeBinary(*static_cast<const IColumnConst &>(nested_column).convertToFullColumn(), ostr, offset, limit);
else
elems[i]->serializeBinary(nested_column, ostr, offset, limit);
}
NativeBlockOutputStream::writeData(*elems[i], real_column.getData().getByPosition(i).column, ostr, offset, limit);
}
/** limit обязательно должен быть в точности равен количеству сериализованных значений.
@ -133,7 +128,7 @@ public:
{
ColumnTuple & real_column = static_cast<ColumnTuple &>(column);
for (size_t i = 0, size = elems.size(); i < size; ++i)
elems[i]->deserializeBinary(*real_column.getData().getByPosition(i).column, istr, limit, avg_value_size_hint);
NativeBlockInputStream::readData(*elems[i], *real_column.getData().getByPosition(i).column, istr, limit);
}
ColumnPtr createColumn() const

View File

@ -4,6 +4,8 @@
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/OwningBufferBlockInputStream.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/DataStreams/FormatFactory.h>
#include <Poco/Timestamp.h>
#include <Poco/File.h>

View File

@ -4,6 +4,7 @@
#include <DB/Dictionaries/IDictionarySource.h>
#include <DB/Dictionaries/DictionaryStructure.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Common/Arena.h>
#include <statdaemons/ext/range.hpp>
#include <vector>
#include <tuple>

View File

@ -9,6 +9,7 @@
#include <DB/Columns/ColumnString.h>
#include <statdaemons/ext/range.hpp>
#include <mysqlxx/Query.h>
#include <mysqlxx/PoolWithFailover.h>
#include <vector>
#include <string>

View File

@ -16,6 +16,8 @@ class Context;
*/
class FunctionFactory : public Singleton<FunctionFactory>
{
friend class StorageSystemFunctions;
private:
typedef IFunction* (*Creator)(const Context & context); /// Не std::function, так как меньше indirection и размер объекта.
std::unordered_map<String, Creator> functions;

View File

@ -11,13 +11,16 @@
#include <DB/Columns/ColumnString.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/Dictionaries.h>
#include <DB/Interpreters/ExternalDictionaries.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <DB/Dictionaries/FlatDictionary.h>
#include <DB/Dictionaries/HashedDictionary.h>
#include <DB/Dictionaries/CacheDictionary.h>
#include <statdaemons/ext/range.hpp>
namespace DB
{

View File

@ -20,6 +20,8 @@
#include <DB/Common/HashTable/Hash.h>
#include <DB/Functions/IFunction.h>
#include <statdaemons/ext/range.hpp>
#include <stats/IntHash.h>
@ -603,6 +605,215 @@ public:
};
struct URLHashImpl
{
static UInt64 apply(const char * data, const std::size_t size)
{
/// do not take last slash, '?' or '#' character into account
if (size > 0 && (data[size - 1] == '/' || data[size - 1] == '?' || data[size - 1] == '#'))
return CityHash64(data, size - 1);
return CityHash64(data, size);
}
};
struct URLHierarchyHashImpl
{
static std::size_t findLevelLength(const UInt64 level, const char * begin, const char * const end)
{
auto pos = begin;
/// Распарсим всё, что идёт до пути
/// Предположим, что протокол уже переведён в нижний регистр.
while (pos < end && ((*pos > 'a' && *pos < 'z') || (*pos > '0' && *pos < '9')))
++pos;
/** Будем вычислять иерархию только для URL-ов, в которых есть протокол, и после него идут два слеша.
* (http, file - подходят, mailto, magnet - не подходят), и после двух слешей ещё хоть что-нибудь есть
* Для остальных просто вернём полный URL как единственный элемент иерархии.
*/
if (pos == begin || pos == end || !(*pos++ == ':' && pos < end && *pos++ == '/' && pos < end && *pos++ == '/' && pos < end))
{
pos = end;
return 0 == level ? pos - begin : 0;
}
/// Доменом для простоты будем считать всё, что после протокола и двух слешей, до следующего слеша или до ? или до #
while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos != end)
++pos;
if (0 == level)
return pos - begin;
UInt64 current_level = 0;
while (current_level != level && pos < end)
{
/// Идём до следующего / или ? или #, пропуская все те, что вначале.
while (pos < end && (*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos == end)
break;
while (pos < end && !(*pos == '/' || *pos == '?' || *pos == '#'))
++pos;
if (pos != end)
++pos;
++current_level;
}
return current_level == level ? pos - begin : 0;
}
static UInt64 apply(const UInt64 level, const char * data, const std::size_t size)
{
return URLHashImpl::apply(data, findLevelLength(level, data, data + size));
}
};
class FunctionURLHash : public IFunction
{
public:
static constexpr auto name = "URLHash";
static IFunction * create(const Context &) { return new FunctionURLHash; }
String getName() const override { return name; }
DataTypePtr getReturnType(const DataTypes & arguments) const override
{
const auto arg_count = arguments.size();
if (arg_count != 1 && arg_count != 2)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(arg_count) + ", should be 1 or 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
const auto first_arg = arguments.front().get();
if (!typeid_cast<const DataTypeString *>(first_arg))
throw Exception{
"Illegal type " + first_arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
if (arg_count == 2)
{
const auto second_arg = arguments.back().get();
if (!typeid_cast<const DataTypeUInt8 *>(second_arg) &&
!typeid_cast<const DataTypeUInt16 *>(second_arg) &&
!typeid_cast<const DataTypeUInt32 *>(second_arg) &&
!typeid_cast<const DataTypeUInt64 *>(second_arg) &&
!typeid_cast<const DataTypeInt8 *>(second_arg) &&
!typeid_cast<const DataTypeInt16 *>(second_arg) &&
!typeid_cast<const DataTypeInt32 *>(second_arg) &&
!typeid_cast<const DataTypeInt64 *>(second_arg))
throw Exception{
"Illegal type " + second_arg->getName() + " of argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
}
return new DataTypeUInt64;
}
void execute(Block & block, const ColumnNumbers & arguments, const size_t result)
{
const auto arg_count = arguments.size();
if (arg_count == 1)
executeSingleArg(block, arguments, result);
else if (arg_count == 2)
executeTwoArgs(block, arguments, result);
else
throw std::logic_error{"got into IFunction::execute with unexpected number of arguments"};
}
private:
void executeSingleArg(Block & block, const ColumnNumbers & arguments, const std::size_t result) const
{
const auto col_untyped = block.getByPosition(arguments.front()).column.get();
if (const auto col_from = typeid_cast<const ColumnString *>(col_untyped))
{
const auto size = col_from->size();
const auto col_to = new ColumnVector<UInt64>{size};
block.getByPosition(result).column = col_to;
const auto & chars = col_from->getChars();
const auto & offsets = col_from->getOffsets();
auto & out = col_to->getData();
for (const auto i : ext::range(0, size))
out[i] = URLHashImpl::apply(
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
}
else if (const auto col_from = typeid_cast<const ColumnConstString *>(col_untyped))
{
block.getByPosition(result).column = new ColumnConstUInt64{
col_from->size(),
URLHashImpl::apply(col_from->getData().data(), col_from->getData().size())
};
}
else
throw Exception{
"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
" of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
void executeTwoArgs(Block & block, const ColumnNumbers & arguments, const std::size_t result) const
{
const auto level_col = block.getByPosition(arguments.back()).column.get();
if (!level_col->isConst())
throw Exception{
"Second argument of function " + getName() + " must be an integral constant",
ErrorCodes::ILLEGAL_COLUMN
};
const auto level = level_col->get64(0);
const auto col_untyped = block.getByPosition(arguments.front()).column.get();
if (const auto col_from = typeid_cast<const ColumnString *>(col_untyped))
{
const auto size = col_from->size();
const auto col_to = new ColumnVector<UInt64>{size};
block.getByPosition(result).column = col_to;
const auto & chars = col_from->getChars();
const auto & offsets = col_from->getOffsets();
auto & out = col_to->getData();
for (const auto i : ext::range(0, size))
out[i] = URLHierarchyHashImpl::apply(level,
reinterpret_cast<const char *>(&chars[i == 0 ? 0 : offsets[i - 1]]),
i == 0 ? offsets[i] - 1 : (offsets[i] - 1 - offsets[i - 1]));
}
else if (const auto col_from = typeid_cast<const ColumnConstString *>(col_untyped))
{
block.getByPosition(result).column = new ColumnConstUInt64{
col_from->size(),
URLHierarchyHashImpl::apply(level, col_from->getData().data(), col_from->getData().size())
};
}
else
throw Exception{
"Illegal column " + block.getByPosition(arguments[0]).column->getName() +
" of argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
}
};
struct NameHalfMD5 { static constexpr auto name = "halfMD5"; };
struct NameSipHash64 { static constexpr auto name = "sipHash64"; };
struct NameCityHash64 { static constexpr auto name = "cityHash64"; };

View File

@ -99,9 +99,6 @@ protected:
*/
Buffer working_buffer;
/// Количество игнорируемых байтов с начальной позиции буфера working_buffer.
size_t working_buffer_offset = 0;
/// Позиция чтения/записи.
Position pos;

View File

@ -7,8 +7,7 @@
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#define DBMS_DEFAULT_BUFFER_SIZE 1048576ULL
#include <DB/Core/Defines.h>
namespace DB
@ -105,7 +104,11 @@ private:
return;
}
int res = posix_memalign(reinterpret_cast<void **>(&new_m_data), alignment, (m_capacity + alignment - 1) / alignment * alignment);
size_t aligned_capacity = (m_capacity + alignment - 1) / alignment * alignment;
m_capacity = aligned_capacity;
m_size = m_capacity;
int res = posix_memalign(reinterpret_cast<void **>(&new_m_data), alignment, m_capacity);
if (0 != res)
DB::throwFromErrno("Cannot allocate memory (posix_memalign)", ErrorCodes::CANNOT_ALLOCATE_MEMORY, res);

View File

@ -1,6 +1,6 @@
#pragma once
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/IO/createReadBufferFromFileBase.h>
#include <DB/IO/CompressedReadBufferBase.h>
#include <DB/IO/UncompressedCache.h>
@ -20,9 +20,11 @@ private:
const std::string path;
UncompressedCache * cache;
size_t buf_size;
size_t estimated_size;
size_t aio_threshold;
/// SharedPtr - для ленивой инициализации (только в случае кэш-промаха).
Poco::SharedPtr<ReadBufferFromFile> file_in;
Poco::SharedPtr<ReadBufferFromFileBase> file_in;
size_t file_pos;
/// Кусок данных из кэша, или кусок считанных данных, который мы положим в кэш.
@ -32,7 +34,7 @@ private:
{
if (!file_in)
{
file_in = new ReadBufferFromFile(path, buf_size);
file_in = createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size);
compressed_in = &*file_in;
}
}
@ -80,8 +82,10 @@ private:
}
public:
CachedCompressedReadBuffer(const std::string & path_, UncompressedCache * cache_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE)
: ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_), file_pos(0)
CachedCompressedReadBuffer(const std::string & path_, UncompressedCache * cache_, size_t estimated_size_,
size_t aio_threshold_, size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE)
: ReadBuffer(nullptr, 0), path(path_), cache(cache_), buf_size(buf_size_),
estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0)
{
}

View File

@ -1,7 +1,10 @@
#pragma once
#include <DB/IO/CompressedReadBufferBase.h>
#include "ReadBufferFromFile.h"
#include <DB/IO/ReadBufferFromFileBase.h>
#include <DB/IO/createReadBufferFromFileBase.h>
#include <memory>
namespace DB
@ -18,7 +21,8 @@ private:
* - file_in смотрит в конец этого блока.
* - size_compressed содержит сжатый размер этого блока.
*/
ReadBufferFromFile file_in;
std::unique_ptr<ReadBufferFromFileBase> p_file_in;
ReadBufferFromFileBase & file_in;
size_t size_compressed = 0;
bool nextImpl()
@ -38,8 +42,10 @@ private:
}
public:
CompressedReadBufferFromFile(const std::string & path, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
: BufferWithOwnMemory<ReadBuffer>(0), file_in(path, buf_size)
CompressedReadBufferFromFile(const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE)
: BufferWithOwnMemory<ReadBuffer>(0),
p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size)),
file_in(*p_file_in)
{
compressed_in = &file_in;
}

View File

@ -118,7 +118,7 @@ public:
while (bytes_copied < n && !eof())
{
size_t bytes_to_copy = std::min(static_cast<size_t>(working_buffer.end() - pos), n - bytes_copied);
std::memcpy(to + bytes_copied, pos, bytes_to_copy);
::memcpy(to + bytes_copied, pos, bytes_to_copy);
pos += bytes_to_copy;
bytes_copied += bytes_to_copy;
}
@ -144,6 +144,10 @@ public:
return read(to, n);
}
protected:
/// Количество игнорируемых байт с начальной позиции буфера working_buffer.
size_t working_buffer_offset = 0;
private:
/** Прочитать следующие данные и заполнить ими буфер.
* Вернуть false в случае конца, true иначе.

View File

@ -1,7 +1,9 @@
#pragma once
#include <DB/IO/ReadBufferFromFileBase.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
#include <DB/Core/Defines.h>
#include <statdaemons/AIO.h>
#include <string>
@ -14,10 +16,10 @@ namespace DB
/** Класс для асинхронного чтения данных.
*/
class ReadBufferAIO : public BufferWithOwnMemory<ReadBuffer>
class ReadBufferAIO : public ReadBufferFromFileBase
{
public:
ReadBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1, mode_t mode_ = 0666,
ReadBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1,
char * existing_memory_ = nullptr);
~ReadBufferAIO() override;
@ -25,23 +27,34 @@ public:
ReadBufferAIO & operator=(const ReadBufferAIO &) = delete;
void setMaxBytes(size_t max_bytes_read_);
off_t seek(off_t off, int whence = SEEK_SET);
off_t getPositionInFile();
std::string getFileName() const noexcept { return filename; }
int getFD() const noexcept { return fd; }
off_t getPositionInFile() override { return first_unread_pos_in_file - (working_buffer.end() - pos); }
std::string getFileName() const override { return filename; }
int getFD() const override { return fd; }
private:
off_t getPositionInFileRelaxed() const noexcept;
bool nextImpl();
///
bool nextImpl() override;
///
off_t doSeek(off_t off, int whence) override;
/// Синхронно читать данные.
void synchronousRead();
/// Получить данные от асинхронного запроса.
void receive();
/// Игнорировать данные от асинхронного запроса.
void skip();
/// Ждать окончания текущей асинхронной задачи.
void waitForAIOCompletion();
/// Менять местами основной и дублирующий буферы.
void swapBuffers() noexcept;
bool waitForAIOCompletion();
/// Подготовить запрос.
void prepare();
/// Подготовить к чтению дублирующий буфер содержащий данные от
/// последнего запроса.
void finalize();
private:
/// Буфер для асинхронных операций чтения данных.
BufferWithOwnMemory<ReadBuffer> fill_buffer;
/// Описание асинхронного запроса на чтение.
iocb request;
std::vector<iocb *> request_ptrs{&request};
std::vector<io_event> events{1};
@ -50,20 +63,41 @@ private:
const std::string filename;
/// Максимальное количество байтов, которое можно прочитать.
size_t max_bytes_read = std::numeric_limits<size_t>::max();
size_t total_bytes_read = 0;
/// Количество запрашиваемых байтов.
size_t requested_byte_count = 0;
off_t pos_in_file = 0;
/// Количество прочитанных байт при последнем запросе.
ssize_t bytes_read = 0;
/// Итоговое количество прочитанных байтов.
size_t total_bytes_read = 0;
/// Позиция первого непрочитанного байта в файле.
off_t first_unread_pos_in_file = 0;
/// Начальная позиция выровненного региона диска, из которого читаются данные.
off_t region_aligned_begin = 0;
/// Левое смещение для выравнения региона диска.
size_t region_left_padding = 0;
/// Размер выровненного региона диска.
size_t region_aligned_size = 0;
/// Файловый дескриптор для чтения.
int fd = -1;
/// Буфер, в который пишутся полученные данные.
Position buffer_begin = nullptr;
/// Асинхронная операция чтения ещё не завершилась.
bool is_pending_read = false;
/// Было получено исключение.
bool got_exception = false;
/// Конец файла достигнут.
bool is_eof = false;
/// Был отправлен хоть один запрос на асинхронную операцию чтения.
/// Был отправлен хоть один запрос на чтение.
bool is_started = false;
/// Является ли операция асинхронной?
bool is_aio = false;
/// Асинхронная операция завершилась неудачно?
bool aio_failed = false;
};
}

View File

@ -0,0 +1,26 @@
#pragma once
#include <string>
#include <fcntl.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
namespace DB
{
class ReadBufferFromFileBase : public BufferWithOwnMemory<ReadBuffer>
{
public:
ReadBufferFromFileBase(size_t buf_size, char * existing_memory, size_t alignment);
virtual ~ReadBufferFromFileBase();
off_t seek(off_t off, int whence = SEEK_SET);
virtual off_t getPositionInFile() = 0;
virtual std::string getFileName() const = 0;
virtual int getFD() const = 0;
protected:
virtual off_t doSeek(off_t off, int whence) = 0;
};
}

View File

@ -8,6 +8,7 @@
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/IO/ReadBufferFromFileBase.h>
#include <DB/IO/ReadBuffer.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/BufferWithOwnMemory.h>
@ -18,12 +19,12 @@ namespace DB
/** Работает с готовым файловым дескриптором. Не открывает и не закрывает файл.
*/
class ReadBufferFromFileDescriptor : public BufferWithOwnMemory<ReadBuffer>
class ReadBufferFromFileDescriptor : public ReadBufferFromFileBase
{
protected:
int fd;
off_t pos_in_file; /// Какому сдвигу в файле соответствует working_buffer.end().
bool nextImpl()
{
size_t bytes_read = 0;
@ -34,7 +35,7 @@ protected:
ssize_t res = ::read(fd, internal_buffer.begin(), internal_buffer.size());
if (!res)
break;
if (-1 == res && errno != EINTR)
throwFromErrno("Cannot read from file " + getFileName(), ErrorCodes::CANNOT_READ_FROM_FILE_DESCRIPTOR);
@ -45,7 +46,10 @@ protected:
pos_in_file += bytes_read;
if (bytes_read)
{
ProfileEvents::increment(ProfileEvents::ReadBufferFromFileDescriptorReadBytes, bytes_read);
working_buffer.resize(bytes_read);
}
else
return false;
@ -53,22 +57,28 @@ protected:
}
/// Имя или описание файла
virtual std::string getFileName()
virtual std::string getFileName() const override
{
return "(fd = " + toString(fd) + ")";
}
public:
ReadBufferFromFileDescriptor(int fd_, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment), fd(fd_), pos_in_file(0) {}
: ReadBufferFromFileBase(buf_size, existing_memory, alignment), fd(fd_), pos_in_file(0) {}
int getFD()
int getFD() const override
{
return fd;
}
off_t getPositionInFile() override
{
return pos_in_file - (working_buffer.end() - pos);
}
private:
/// Если offset такой маленький, что мы не выйдем за пределы буфера, настоящий seek по файлу не делается.
off_t seek(off_t offset, int whence = SEEK_SET)
off_t doSeek(off_t offset, int whence) override
{
off_t new_pos = offset;
if (whence == SEEK_CUR)
@ -98,11 +108,6 @@ public:
return res;
}
}
off_t getPositionInFile()
{
return pos_in_file - (working_buffer.end() - pos);
}
};
}

View File

@ -70,6 +70,6 @@ public:
}
};
typedef Poco::SharedPtr<UncompressedCache> UncompressedCachePtr;
typedef std::shared_ptr<UncompressedCache> UncompressedCachePtr;
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <DB/IO/WriteBufferFromFileBase.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
#include <DB/Core/Defines.h>
@ -8,15 +9,13 @@
#include <string>
#include <unistd.h>
#include <fcntl.h>
#include <sys/uio.h>
namespace DB
{
/** Класс для асинхронной записи данных.
* Все размеры и смещения должны быть кратны DEFAULT_AIO_FILE_BLOCK_SIZE байтам.
*/
class WriteBufferAIO : public BufferWithOwnMemory<WriteBuffer>
class WriteBufferAIO : public WriteBufferFromFileBase
{
public:
WriteBufferAIO(const std::string & filename_, size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE, int flags_ = -1, mode_t mode_ = 0666,
@ -26,40 +25,45 @@ public:
WriteBufferAIO(const WriteBufferAIO &) = delete;
WriteBufferAIO & operator=(const WriteBufferAIO &) = delete;
off_t seek(off_t off, int whence = SEEK_SET);
off_t getPositionInFile();
void truncate(off_t length = 0);
void sync();
std::string getFileName() const noexcept { return filename; }
int getFD() const noexcept { return fd; }
off_t getPositionInFile() override;
void truncate(off_t length = 0) override;
void sync() override;
std::string getFileName() const override { return filename; }
int getFD() const override { return fd; }
private:
///
void nextImpl() override;
///
off_t doSeek(off_t off, int whence) override;
/// Если в буфере ещё остались данные - запишем их.
void flush();
///
void nextImpl();
/// Ждать окончания текущей асинхронной задачи.
void waitForAIOCompletion();
/// Менять местами основной и дублирующий буферы.
void swapBuffers() noexcept;
bool waitForAIOCompletion();
/// Подготовить асинхронный запрос.
void prepare();
///
void finalize();
private:
/// Буфер для асинхронных операций записи данных.
BufferWithOwnMemory<WriteBuffer> flush_buffer;
/// Описание асинхронного запроса на запись.
iocb request;
std::vector<iocb *> request_ptrs{&request};
std::vector<io_event> events{1};
AIOContext aio_context{1};
iovec iov[3];
Memory memory_page{DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE};
const std::string filename;
/// Количество байтов, которые будут записаны на диск.
off_t bytes_to_write = 0;
/// Количество записанных байт при последнем запросе.
off_t bytes_written = 0;
/// Количество нулевых байтов, которые надо отрезать c конца файла
/// после завершения операции записи данных.
off_t truncation_count = 0;
/// Текущая позиция в файле.
@ -67,15 +71,21 @@ private:
/// Максимальная достигнутая позиция в файле.
off_t max_pos_in_file = 0;
/// Начальная позиция выровненного региона диска, в который записываются данные.
off_t region_aligned_begin = 0;
/// Размер выровненного региона диска.
size_t region_aligned_size = 0;
/// Файловый дескриптор для записи.
int fd = -1;
/// Файловый дескриптор для чтения. Употребляется для невыровненных записей.
int fd2 = -1;
/// Асинхронная операция записи ещё не завершилась.
/// Буфер данных, которые хотим записать на диск.
Position buffer_begin = nullptr;
/// Асинхронная операция записи ещё не завершилась?
bool is_pending_write = false;
/// Было получено исключение.
bool got_exception = false;
/// Асинхронная операция завершилась неудачно?
bool aio_failed = false;
};
}

View File

@ -0,0 +1,28 @@
#pragma once
#include <string>
#include <fcntl.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/BufferWithOwnMemory.h>
namespace DB
{
class WriteBufferFromFileBase : public BufferWithOwnMemory<WriteBuffer>
{
public:
WriteBufferFromFileBase(size_t buf_size, char * existing_memory, size_t alignment);
virtual ~WriteBufferFromFileBase();
off_t seek(off_t off, int whence = SEEK_SET);
virtual off_t getPositionInFile() = 0;
virtual void truncate(off_t length) = 0;
virtual void sync() = 0;
virtual std::string getFileName() const = 0;
virtual int getFD() const = 0;
protected:
virtual off_t doSeek(off_t off, int whence) = 0;
};
}

View File

@ -5,7 +5,9 @@
#include <DB/Core/Exception.h>
#include <DB/Core/ErrorCodes.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/IO/WriteBufferFromFileBase.h>
#include <DB/IO/WriteBuffer.h>
#include <DB/IO/WriteHelpers.h>
#include <DB/IO/BufferWithOwnMemory.h>
@ -16,11 +18,11 @@ namespace DB
/** Работает с готовым файловым дескриптором. Не открывает и не закрывает файл.
*/
class WriteBufferFromFileDescriptor : public BufferWithOwnMemory<WriteBuffer>
class WriteBufferFromFileDescriptor : public WriteBufferFromFileBase
{
protected:
int fd;
void nextImpl()
{
if (!offset())
@ -29,6 +31,8 @@ protected:
size_t bytes_written = 0;
while (bytes_written != offset())
{
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWrite);
ssize_t res = ::write(fd, working_buffer.begin() + bytes_written, offset() - bytes_written);
if ((-1 == res || 0 == res) && errno != EINTR)
@ -37,17 +41,19 @@ protected:
if (res > 0)
bytes_written += res;
}
ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteBytes, bytes_written);
}
/// Имя или описание файла
virtual std::string getFileName()
virtual std::string getFileName() const override
{
return "(fd = " + toString(fd) + ")";
}
public:
WriteBufferFromFileDescriptor(int fd_ = -1, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr, size_t alignment = 0)
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment), fd(fd_) {}
: WriteBufferFromFileBase(buf_size, existing_memory, alignment), fd(fd_) {}
/** Можно вызывать для инициализации, если нужный fd не был передан в конструктор.
* Менять fd во время работы нельзя.
@ -69,27 +75,24 @@ public:
}
}
int getFD()
int getFD() const override
{
return fd;
}
off_t seek(off_t offset, int whence = SEEK_SET)
off_t getPositionInFile() override
{
off_t res = lseek(fd, offset, whence);
if (-1 == res)
throwFromErrno("Cannot seek through file " + getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
return res;
return seek(0, SEEK_CUR);
}
void truncate(off_t length = 0)
void truncate(off_t length = 0) override
{
int res = ftruncate(fd, length);
if (-1 == res)
throwFromErrno("Cannot truncate file " + getFileName(), ErrorCodes::CANNOT_TRUNCATE_FILE);
}
void sync()
void sync() override
{
/// Если в буфере ещё остались данные - запишем их.
next();
@ -99,6 +102,15 @@ public:
if (-1 == res)
throwFromErrno("Cannot fsync " + getFileName(), ErrorCodes::CANNOT_FSYNC);
}
private:
off_t doSeek(off_t offset, int whence) override
{
off_t res = lseek(fd, offset, whence);
if (-1 == res)
throwFromErrno("Cannot seek through file " + getFileName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE);
return res;
}
};
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <DB/IO/ReadBufferFromFileBase.h>
#include <string>
namespace DB
{
/** Создать объект для чтения данных из файла.
* estimated_size - количество байтов, которые надо читать
* aio_threshold - минимальное количество байт для асинхронных операций чтения
*
* Если aio_threshold = 0 или estimated_size < aio_threshold, операции чтения выполняются синхронно.
* В противном случае операции чтения выполняются асинхронно.
*/
ReadBufferFromFileBase * createReadBufferFromFileBase(const std::string & filename_,
size_t estimated_size,
size_t aio_threshold,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
int flags_ = -1,
char * existing_memory_ = nullptr,
size_t alignment = 0);
}

View File

@ -0,0 +1,25 @@
#pragma once
#include <DB/IO/WriteBufferFromFileBase.h>
#include <string>
namespace DB
{
/** Создать объект для записи данных в файл.
* estimated_size - количество байтов, которые надо записать
* aio_threshold - минимальное количество байт для асинхронных операций записи
*
* Если aio_threshold = 0 или estimated_size < aio_threshold, операции записи выполняются синхронно.
* В противном случае операции записи выполняются асинхронно.
*/
WriteBufferFromFileBase * createWriteBufferFromFileBase(const std::string & filename_,
size_t estimated_size,
size_t aio_threshold,
size_t buffer_size_ = DBMS_DEFAULT_BUFFER_SIZE,
int flags_ = -1,
mode_t mode = 0666,
char * existing_memory_ = nullptr,
size_t alignment = 0);
}

View File

@ -2,6 +2,7 @@
#include <mutex>
#include <memory>
#include <functional>
#include <Yandex/logger_useful.h>
#include <statdaemons/threadpool.hpp>
@ -523,6 +524,7 @@ struct AggregatedDataVariants : private boost::noncopyable
AggregatedDataVariants() : aggregates_pools(1, new Arena), aggregates_pool(&*aggregates_pools.back()) {}
bool empty() const { return type == Type::EMPTY; }
void invalidate() { type = Type::EMPTY; }
~AggregatedDataVariants();
@ -682,7 +684,8 @@ public:
: key_names(key_names_), aggregates(aggregates_), aggregates_size(aggregates.size()),
overflow_row(overflow_row_),
max_rows_to_group_by(max_rows_to_group_by_), group_by_overflow_mode(group_by_overflow_mode_),
compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_)
compiler(compiler_), min_count_to_compile(min_count_to_compile_), group_by_two_level_threshold(group_by_two_level_threshold_),
isCancelled([]() { return false; })
{
std::sort(key_names.begin(), key_names.end());
key_names.erase(std::unique(key_names.begin(), key_names.end()), key_names.end());
@ -723,6 +726,12 @@ public:
*/
void mergeStream(BlockInputStreamPtr stream, AggregatedDataVariants & result, size_t max_threads);
using CancellationHook = std::function<bool()>;
/** Установить функцию, которая проверяет, можно ли прервать текущую задачу.
*/
void setCancellationHook(const CancellationHook cancellation_hook);
/// Для IBlockInputStream.
String getID() const;
@ -786,6 +795,9 @@ protected:
*/
size_t group_by_two_level_threshold;
/// Возвращает true, если можно прервать текущую задачу.
CancellationHook isCancelled;
/** Если заданы только имена столбцов (key_names, а также aggregates[i].column_name), то вычислить номера столбцов.
* Сформировать блок - пример результата.
*/

View File

@ -1,44 +1,45 @@
#pragma once
#include <map>
#include <set>
#include <Poco/SharedPtr.h>
#include <Poco/Mutex.h>
#include <Yandex/logger_useful.h>
#include <functional>
#include <memory>
#include <DB/Core/Types.h>
#include <DB/Core/NamesAndTypes.h>
#include <DB/Common/Macros.h>
#include <DB/IO/UncompressedCache.h>
#include <DB/Storages/MarkCache.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/Storages/IStorage.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/DataTypes/DataTypeFactory.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
#include <DB/Storages/MergeTree/MergeList.h>
#include <DB/TableFunctions/TableFunctionFactory.h>
#include <DB/Interpreters/Settings.h>
#include <DB/Interpreters/Users.h>
#include <DB/Interpreters/Quota.h>
#include <DB/Interpreters/Dictionaries.h>
#include <DB/Interpreters/ExternalDictionaries.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Interpreters/Cluster.h>
#include <DB/Interpreters/InterserverIOHandler.h>
#include <DB/Interpreters/Compiler.h>
#include <DB/Client/ConnectionPool.h>
#include <statdaemons/ConfigProcessor.h>
#include <zkutil/ZooKeeper.h>
#include <DB/Storages/IStorage.h>
#include <DB/IO/CompressedStream.h>
#include <Poco/Net/IPAddress.h>
namespace zkutil
{
class ZooKeeper;
}
namespace DB
{
class ContextShared;
class QuotaForIntervals;
class TableFunctionFactory;
class AggregateFunctionFactory;
class FormatFactory;
class Dictionaries;
class ExternalDictionaries;
class InterserverIOHandler;
class BackgroundProcessingPool;
class MergeList;
class Cluster;
class Compiler;
class MarkCache;
class UncompressedCache;
class ProcessList;
class ProcessListElement;
class Macros;
class Progress;
using Poco::SharedPtr;
/// имя таблицы -> таблица
typedef std::map<String, StoragePtr> Tables;
@ -53,112 +54,6 @@ typedef std::pair<String, String> DatabaseAndTableName;
typedef std::map<DatabaseAndTableName, std::set<DatabaseAndTableName>> ViewDependencies;
typedef std::vector<DatabaseAndTableName> Dependencies;
/** Набор известных объектов, которые могут быть использованы в запросе.
* Разделяемая часть. Порядок членов (порядок их уничтожения) очень важен.
*/
struct ContextShared
{
Logger * log = &Logger::get("Context"); /// Логгер.
struct AfterDestroy
{
Logger * log;
AfterDestroy(Logger * log_) : log(log_) {}
~AfterDestroy()
{
#ifndef DBMS_CLIENT
LOG_INFO(log, "Uninitialized shared context.");
#endif
}
} after_destroy {log};
mutable Poco::Mutex mutex; /// Для доступа и модификации разделяемых объектов.
mutable zkutil::ZooKeeperPtr zookeeper; /// Клиент для ZooKeeper.
String interserver_io_host; /// Имя хоста по которым это сервер доступен для других серверов.
int interserver_io_port; /// и порт,
String path; /// Путь к директории с данными, со слешем на конце.
String tmp_path; /// Путь ко временным файлам, возникающим при обработке запроса.
Databases databases; /// Список БД и таблиц в них.
TableFunctionFactory table_function_factory; /// Табличные функции.
AggregateFunctionFactory aggregate_function_factory; /// Агрегатные функции.
DataTypeFactory data_type_factory; /// Типы данных.
FormatFactory format_factory; /// Форматы.
mutable SharedPtr<Dictionaries> dictionaries; /// Словари Метрики. Инициализируются лениво.
mutable SharedPtr<ExternalDictionaries> external_dictionaries;
Users users; /// Известные пользователи.
Quotas quotas; /// Известные квоты на использование ресурсов.
mutable UncompressedCachePtr uncompressed_cache; /// Кэш разжатых блоков.
mutable MarkCachePtr mark_cache; /// Кэш засечек в сжатых файлах.
ProcessList process_list; /// Исполняющиеся в данный момент запросы.
MergeList merge_list; /// Список выполняемых мерджей (для (Replicated)?MergeTree)
ViewDependencies view_dependencies; /// Текущие зависимости
ConfigurationPtr users_config; /// Конфиг с секциями users, profiles и quotas.
InterserverIOHandler interserver_io_handler; /// Обработчик для межсерверной передачи данных.
BackgroundProcessingPoolPtr background_pool; /// Пул потоков для фоновой работы, выполняемой таблицами.
Macros macros; /// Подстановки из конфига.
std::unique_ptr<Compiler> compiler; /// Для динамической компиляции частей запроса, при необходимости.
/// Кластеры для distributed таблиц
/// Создаются при создании Distributed таблиц, так как нужно дождаться пока будут выставлены Settings
Poco::SharedPtr<Clusters> clusters;
bool shutdown_called = false;
~ContextShared()
{
#ifndef DBMS_CLIENT
LOG_INFO(log, "Uninitializing shared context.");
#endif
try
{
shutdown();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
/** Выполнить сложную работу по уничтожению объектов заранее.
*/
void shutdown()
{
if (shutdown_called)
return;
shutdown_called = true;
/** В этот момент, некоторые таблицы могут иметь потоки,
* которые модифицируют список таблиц, и блокируют наш mutex (см. StorageChunkMerger).
* Чтобы корректно их завершить, скопируем текущий список таблиц,
* и попросим их всех закончить свою работу.
* Потом удалим все объекты с таблицами.
*/
Databases current_databases;
{
Poco::ScopedLock<Poco::Mutex> lock(mutex);
current_databases = databases;
}
for (Databases::iterator it = current_databases.begin(); it != current_databases.end(); ++it)
for (Tables::iterator jt = it->second.begin(); jt != it->second.end(); ++jt)
jt->second->shutdown();
{
Poco::ScopedLock<Poco::Mutex> lock(mutex);
databases.clear();
}
}
};
/** Набор известных объектов, которые могут быть использованы в запросе.
* Состоит из разделяемой части (всегда общей для всех сессий и запросов)
@ -169,31 +64,37 @@ struct ContextShared
class Context
{
private:
typedef SharedPtr<ContextShared> Shared;
Shared shared = new ContextShared;
typedef std::shared_ptr<ContextShared> Shared;
Shared shared;
String user; /// Текущий пользователь.
Poco::Net::IPAddress ip_address; /// IP-адрес, с которого задан запрос.
QuotaForIntervalsPtr quota = new QuotaForIntervals; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
std::shared_ptr<QuotaForIntervals> quota; /// Текущая квота. По-умолчанию - пустая квота, которая ничего не ограничивает.
String current_database; /// Текущая БД.
String current_query_id; /// Id текущего запроса.
NamesAndTypesList columns; /// Столбцы текущей обрабатываемой таблицы.
Settings settings; /// Настройки выполнения запроса.
using ProgressCallback = std::function<void(const Progress & progress)>;
ProgressCallback progress_callback; /// Колбек для отслеживания прогресса выполнения запроса.
ProcessList::Element * process_list_elem = nullptr; /// Для отслеживания общего количества потраченных на запрос ресурсов.
ProcessListElement * process_list_elem = nullptr; /// Для отслеживания общего количества потраченных на запрос ресурсов.
String default_format; /// Формат, используемый, если сервер сам форматирует данные, и если в запросе не задан FORMAT.
/// То есть, используется в HTTP-интерфейсе. Может быть не задан - тогда используется некоторый глобальный формат по-умолчанию.
String default_format; /// Формат, используемый, если сервер сам форматирует данные, и если в запросе не задан FORMAT.
/// То есть, используется в HTTP-интерфейсе. Может быть не задан - тогда используется некоторый глобальный формат по-умолчанию.
Tables external_tables; /// Временные таблицы.
Context * session_context = nullptr; /// Контекст сессии или nullptr, если его нет. (Возможно, равен this.)
Context * global_context = nullptr; /// Глобальный контекст или nullptr, если его нет. (Возможно, равен this.)
public:
Context();
~Context();
String getPath() const;
String getTemporaryPath() const;
void setPath(const String & path);
void setTemporaryPath(const String & path);
using ConfigurationPtr = Poco::AutoPtr<Poco::Util::AbstractConfiguration>;
/** Забрать список пользователей, квот и профилей настроек из этого конфига.
* Список пользователей полностью заменяется.
* Накопленные значения у квоты не сбрасываются, если квота не удалена.
@ -256,16 +157,16 @@ public:
/// Установить настройку по имени. Прочитать значение в текстовом виде из строки (например, из конфига, или из параметра URL).
void setSetting(const String & name, const std::string & value);
const TableFunctionFactory & getTableFunctionFactory() const { return shared->table_function_factory; }
const AggregateFunctionFactory & getAggregateFunctionFactory() const { return shared->aggregate_function_factory; }
const DataTypeFactory & getDataTypeFactory() const { return shared->data_type_factory; }
const FormatFactory & getFormatFactory() const { return shared->format_factory; }
const TableFunctionFactory & getTableFunctionFactory() const;
const AggregateFunctionFactory & getAggregateFunctionFactory() const;
const DataTypeFactory & getDataTypeFactory() const;
const FormatFactory & getFormatFactory() const;
const Dictionaries & getDictionaries() const;
const ExternalDictionaries & getExternalDictionaries() const;
void tryCreateDictionaries() const;
void tryCreateExternalDictionaries() const;
InterserverIOHandler & getInterserverIOHandler() { return shared->interserver_io_handler; }
InterserverIOHandler & getInterserverIOHandler();
/// Как другие серверы могут обратиться к этому для скачивания реплицируемых данных.
void setInterserverIOAddress(const String & host, UInt16 port);
@ -275,11 +176,11 @@ public:
ASTPtr getCreateQuery(const String & database_name, const String & table_name) const;
/// Для методов ниже может быть необходимо захватывать mutex самостоятельно.
Poco::Mutex & getMutex() const { return shared->mutex; }
Poco::Mutex & getMutex() const;
/// Метод getDatabases не потокобезопасен. При работе со списком БД и таблиц, вы должны захватить mutex.
const Databases & getDatabases() const { return shared->databases; }
Databases & getDatabases() { return shared->databases; }
const Databases & getDatabases() const;
Databases & getDatabases();
/// При работе со списком столбцов, используйте локальный контекст, чтобы никто больше его не менял.
const NamesAndTypesList & getColumns() const { return columns; }
@ -302,28 +203,28 @@ public:
/** Устанавливается в executeQuery и InterpreterSelectQuery. Затем используется в IProfilingBlockInputStream,
* чтобы обновлять и контролировать информацию об общем количестве потраченных на запрос ресурсов.
*/
void setProcessListElement(ProcessList::Element * elem);
void setProcessListElement(ProcessListElement * elem);
/// Может вернуть nullptr, если запрос не был вставлен в ProcessList.
ProcessList::Element * getProcessListElement();
ProcessListElement * getProcessListElement();
/// Список всех запросов.
ProcessList & getProcessList() { return shared->process_list; }
const ProcessList & getProcessList() const { return shared->process_list; }
ProcessList & getProcessList();
const ProcessList & getProcessList() const;
MergeList & getMergeList() { return shared->merge_list; }
const MergeList & getMergeList() const { return shared->merge_list; }
MergeList & getMergeList();
const MergeList & getMergeList() const;
/// Создать кэш разжатых блоков указанного размера. Это можно сделать только один раз.
void setUncompressedCache(size_t max_size_in_bytes);
UncompressedCachePtr getUncompressedCache() const;
std::shared_ptr<UncompressedCache> getUncompressedCache() const;
void setZooKeeper(zkutil::ZooKeeperPtr zookeeper);
void setZooKeeper(std::shared_ptr<zkutil::ZooKeeper> zookeeper);
/// Если в момент вызова текущая сессия просрочена, синхронно создает и возвращает новую вызовом startNewSession().
zkutil::ZooKeeperPtr getZooKeeper() const;
std::shared_ptr<zkutil::ZooKeeper> getZooKeeper() const;
/// Создать кэш засечек указанного размера. Это можно сделать только один раз.
void setMarkCache(size_t cache_size_in_bytes);
MarkCachePtr getMarkCache() const;
std::shared_ptr<MarkCache> getMarkCache() const;
BackgroundProcessingPool & getBackgroundPool();
@ -340,7 +241,10 @@ public:
Compiler & getCompiler();
void shutdown() { shared->shutdown(); }
/// Позволяет выбрать метод сжатия по условиям, описанным в конфигурационном файле.
CompressionMethod chooseCompressionMethod(size_t part_size, double part_size_ratio) const;
void shutdown();
private:
const Dictionaries & getDictionariesImpl(bool throw_on_error) const;

View File

@ -1,5 +1,6 @@
#pragma once
#include <thread>
#include <Yandex/MultiVersion.h>
#include <Yandex/logger_useful.h>
#include <statdaemons/RegionsHierarchies.h>

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/DataTypes/IDataType.h>
#include <DB/DataStreams/IBlockInputStream.h>
#include <DB/Interpreters/Settings.h>
#include <DB/Core/Names.h>
#include <DB/Core/ColumnWithNameAndType.h>
@ -67,7 +68,7 @@ public:
NameSet array_joined_columns;
/// Для JOIN
Join * join = nullptr;
const Join * join = nullptr;
NamesAndTypesList columns_added_by_join;
/// Для PROJECT.
@ -131,7 +132,7 @@ public:
return a;
}
static ExpressionAction ordinaryJoin(Join * join_, const NamesAndTypesList & columns_added_by_join_)
static ExpressionAction ordinaryJoin(const Join * join_, const NamesAndTypesList & columns_added_by_join_)
{
ExpressionAction a;
a.type = JOIN;
@ -152,6 +153,7 @@ private:
std::vector<ExpressionAction> getPrerequisites(Block & sample_block);
void prepare(Block & sample_block);
void execute(Block & block) const;
void executeOnTotals(Block & block) const;
};
@ -165,20 +167,18 @@ public:
ExpressionActions(const NamesAndTypesList & input_columns_, const Settings & settings_)
: input_columns(input_columns_), settings(settings_)
{
for (NamesAndTypesList::iterator it = input_columns.begin(); it != input_columns.end(); ++it)
{
sample_block.insert(ColumnWithNameAndType(nullptr, it->type, it->name));
}
for (const auto & input_elem : input_columns)
sample_block.insert(ColumnWithNameAndType(nullptr, input_elem.type, input_elem.name));
}
/// Для константных столбцов в input_columns_ могут содержаться сами столбцы.
ExpressionActions(const ColumnsWithNameAndType & input_columns_, const Settings & settings_)
: settings(settings_)
: settings(settings_)
{
for (ColumnsWithNameAndType::const_iterator it = input_columns_.begin(); it != input_columns_.end(); ++it)
for (const auto & input_elem : input_columns_)
{
input_columns.emplace_back(it->name, it->type);
sample_block.insert(*it);
input_columns.emplace_back(input_elem.name, input_elem.type);
sample_block.insert(input_elem);
}
}
@ -227,6 +227,11 @@ public:
/// Выполнить выражение над блоком. Блок должен содержать все столбцы , возвращаемые getRequiredColumns.
void execute(Block & block) const;
/** Выполнить выражение над блоком тотальных значений.
* Почти не отличается от execute. Разница лишь при выполнении JOIN-а.
*/
void executeOnTotals(Block & block) const;
/// Получить блок-образец, содержащий имена и типы столбцов результата.
const Block & getSampleBlock() const { return sample_block; }
@ -236,6 +241,8 @@ public:
static std::string getSmallestColumn(const NamesAndTypesList & columns);
BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin(size_t max_block_size) const;
private:
NamesAndTypesList input_columns;
Actions actions;

View File

@ -61,10 +61,10 @@ public:
}
/// Есть ли в выражении агрегатные функции или секция GROUP BY или HAVING.
bool hasAggregation() { return has_aggregation; }
bool hasAggregation() const { return has_aggregation; }
/// Получить список ключей агрегирования и описаний агрегатных функций, если в запросе есть GROUP BY.
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates);
void getAggregateInfo(Names & key_names, AggregateDescriptions & aggregates) const;
/** Получить набор столбцов, которых достаточно прочитать из таблицы для вычисления выражения.
* Не учитываются столбцы, добавляемые из другой таблицы путём JOIN-а.
@ -97,7 +97,7 @@ public:
void appendSelect(ExpressionActionsChain & chain, bool only_types);
bool appendOrderBy(ExpressionActionsChain & chain, bool only_types);
/// Удаляет все столбцы кроме выбираемых SELECT, упорядочивает оставшиеся столбцы и переименовывает их в алиасы.
void appendProjectResult(ExpressionActionsChain & chain, bool only_types);
void appendProjectResult(ExpressionActionsChain & chain, bool only_types) const;
/// Если ast не запрос SELECT, просто получает все действия для вычисления выражения.
/// Если project_result, в выходном блоке останутся только вычисленные значения в нужном порядке, переименованные в алиасы.
@ -117,7 +117,7 @@ public:
/** Таблицы, которые надо будет отправить на удалённые серверы при распределённой обработке запроса.
*/
const Tables & getExternalTables() { return external_tables; }
const Tables & getExternalTables() const { return external_tables; }
/// Если ast - запрос SELECT, получает имена (алиасы) и типы столбцов из секции SELECT.
Block getSelectSampleBlock();
@ -238,9 +238,9 @@ private:
void getArrayJoinedColumns();
void getArrayJoinedColumnsImpl(ASTPtr ast);
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions);
void addMultipleArrayJoinAction(ExpressionActionsPtr & actions) const;
void addJoinAction(ExpressionActionsPtr & actions, bool only_types);
void addJoinAction(ExpressionActionsPtr & actions, bool only_types) const;
struct ScopeStack;
void getActionsImpl(ASTPtr ast, bool no_subqueries, bool only_consts, ScopeStack & actions_stack);
@ -270,11 +270,10 @@ private:
StoragePtr getTable();
/// columns - столбцы, присутствующие до начала преобразований.
void initChain(ExpressionActionsChain & chain, NamesAndTypesList & columns);
void initChain(ExpressionActionsChain & chain, const NamesAndTypesList & columns) const;
void assertSelect();
void assertAggregation();
void assertArrayJoin();
void assertSelect() const;
void assertAggregation() const;
/** Создать Set из явного перечисления значений в запросе.
* Если create_ordered_set = true - создать структуру данных, подходящую для использования индекса.

View File

@ -50,9 +50,9 @@ private:
String from; /// Для FETCH PARTITION - путь в ZK к шарду, с которого скачивать партицию.
static PartitionCommand dropPartition(const Field & partition, bool detach)
static PartitionCommand dropPartition(const Field & partition, bool detach, bool unreplicated)
{
return {DROP_PARTITION, partition, detach};
return {DROP_PARTITION, partition, detach, unreplicated};
}
static PartitionCommand attachPartition(const Field & partition, bool unreplicated, bool part)

View File

@ -6,6 +6,7 @@
#include <DB/Interpreters/Context.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/DataStreams/BlockIO.h>
#include <DB/DataStreams/FormatFactory.h>
#include <DB/DataStreams/copyData.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>

View File

@ -15,14 +15,16 @@ class InterpreterOptimizeQuery
{
public:
InterpreterOptimizeQuery(ASTPtr query_ptr_, Context & context_)
: query_ptr(query_ptr_), context(context_) {}
: query_ptr(query_ptr_), context(context_)
{
}
void execute()
{
const ASTOptimizeQuery & ast = typeid_cast<const ASTOptimizeQuery &>(*query_ptr);
StoragePtr table = context.getTable(ast.database, ast.table);
auto table_lock = table->lockStructure(true);
table->optimize();
table->optimize(context.getSettings());
}
private:

View File

@ -63,7 +63,8 @@ public:
*/
BlockInputStreamPtr execute();
/// Выполнить запрос без объединения потоков.
/** Выполнить запрос без объединения потоков, если это возможно.
*/
const BlockInputStreams & executeWithoutUnion();
/** Выполнить запрос, записать результат в нужном формате в buf.
@ -122,6 +123,8 @@ private:
void executeDistinct( BlockInputStreams & streams, bool before_order, Names columns);
void executeSubqueriesInSetsAndJoins(BlockInputStreams & streams, SubqueriesForSets & subqueries_for_sets);
void ignoreWithTotals() { query.group_by_with_totals = false; }
ASTPtr query_ptr;
ASTSelectQuery & query;
Context context;
@ -142,6 +145,9 @@ private:
StoragePtr storage;
IStorage::TableStructureReadLockPtr table_lock;
/// Выполнить объединение потоков внутри запроса SELECT?
bool union_within_single_query = false;
Logger * log;
};

View File

@ -19,7 +19,7 @@ namespace DB
/** Структура данных для реализации JOIN-а.
* По сути, хэш-таблица: ключи -> строки присоединяемой таблицы.
*
* JOIN-ы бывают четырёх типов: ANY/ALL x LEFT/INNER.
* JOIN-ы бывают восьми типов: ANY/ALL x LEFT/INNER/RIGHT/FULL.
*
* Если указано ANY - выбрать из "правой" таблицы только одну, первую попавшуюся строку, даже если там более одной соответствующей строки.
* Если указано ALL - обычный вариант JOIN-а, при котором строки могут размножаться по числу соответствующих строк "правой" таблицы.
@ -27,6 +27,12 @@ namespace DB
*
* Если указано INNER - оставить только строки, для которых есть хотя бы одна строка "правой" таблицы.
* Если указано LEFT - в случае, если в "правой" таблице нет соответствующей строки, заполнить её значениями "по-умолчанию".
* Если указано RIGHT - выполнить так же, как INNER, запоминая те строки из правой таблицы, которые были присоединены,
* в конце добавить строки из правой таблицы, которые не были присоединены, подставив в качестве значений для левой таблицы, значения "по-умолчанию".
* Если указано FULL - выполнить так же, как LEFT, запоминая те строки из правой таблицы, которые были присоединены,
* в конце добавить строки из правой таблицы, которые не были присоединены, подставив в качестве значений для левой таблицы, значения "по-умолчанию".
*
* То есть, LEFT и RIGHT JOIN-ы не являются симметричными с точки зрения реализации.
*
* Все соединения делаются по равенству кортежа столбцов "ключей" (эквисоединение).
* Неравенства и прочие условия не поддерживаются.
@ -77,11 +83,26 @@ public:
*/
void joinBlock(Block & block) const;
/** Запомнить тотальные значения для последующего использования.
*/
void setTotals(const Block & block) { totals = block; }
bool hasTotals() const { return totals; };
void joinTotals(Block & block) const;
/** Для RIGHT и FULL JOIN-ов.
* Поток, в котором значения по-умолчанию из левой таблицы соединены с неприсоединёнными ранее строками из правой таблицы.
* Использовать только после того, как были сделаны все вызовы joinBlock.
*/
BlockInputStreamPtr createStreamWithNonJoinedRows(Block & left_sample_block, size_t max_block_size) const;
/// Считает суммарное число ключей во всех Join'ах
size_t getTotalRowCount() const;
/// Считает суммарный размер в байтах буфферов всех Join'ов + размер string_pool'а
size_t getTotalByteCount() const;
ASTJoin::Kind getKind() const { return kind; }
/// Ссылка на строку в блоке.
struct RowRef
@ -102,40 +123,65 @@ public:
RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_) {}
};
/** Добавляет или не добавляет флаг - был ли элемент использован.
* Для реализации RIGHT и FULL JOIN-ов.
* NOTE: Можно сохранять флаг в один из бит указателя block или номера row_num.
*/
template <bool enable, typename Base>
struct WithUsedFlag;
template <typename Base>
struct WithUsedFlag<true, Base> : Base
{
mutable bool used = false;
using Base::Base;
using Base_t = Base;
void setUsed() const { used = true; } /// Может выполняться из разных потоков.
bool getUsed() const { return used; }
};
template <typename Base>
struct WithUsedFlag<false, Base> : Base
{
using Base::Base;
using Base_t = Base;
void setUsed() const {}
bool getUsed() const { return true; }
};
/** Разные структуры данных, которые могут использоваться для соединения.
*/
struct MapsAny
template <typename Mapped>
struct MapsTemplate
{
/// Специализация для случая, когда есть один числовой ключ.
typedef HashMap<UInt64, RowRef, HashCRC32<UInt64>> MapUInt64;
typedef HashMap<UInt64, Mapped, HashCRC32<UInt64>> MapUInt64;
/// Специализация для случая, когда есть один строковый ключ.
typedef HashMapWithSavedHash<StringRef, RowRef> MapString;
typedef HashMapWithSavedHash<StringRef, Mapped> MapString;
/** Сравнивает 128 битные хэши.
* Если все ключи фиксированной длины, влезающие целиком в 128 бит, то укладывает их без изменений в 128 бит.
* Иначе - вычисляет SipHash от набора из всех ключей.
* (При этом, строки, содержащие нули посередине, могут склеиться.)
*/
typedef HashMap<UInt128, RowRef, UInt128HashCRC32> MapHashed;
typedef HashMap<UInt128, Mapped, UInt128HashCRC32> MapHashed;
std::unique_ptr<MapUInt64> key64;
std::unique_ptr<MapString> key_string;
std::unique_ptr<MapHashed> hashed;
};
struct MapsAll
{
typedef HashMap<UInt64, RowRefList, HashCRC32<UInt64>> MapUInt64;
typedef HashMapWithSavedHash<StringRef, RowRefList> MapString;
typedef HashMap<UInt128, RowRefList, UInt128HashCRC32> MapHashed;
std::unique_ptr<MapUInt64> key64;
std::unique_ptr<MapString> key_string;
std::unique_ptr<MapHashed> hashed;
};
using MapsAny = MapsTemplate<WithUsedFlag<false, RowRef>>;
using MapsAll = MapsTemplate<WithUsedFlag<false, RowRefList>>;
using MapsAnyFull = MapsTemplate<WithUsedFlag<true, RowRef>>;
using MapsAllFull = MapsTemplate<WithUsedFlag<true, RowRefList>>;
private:
friend class NonJoinedBlockInputStream;
ASTJoin::Kind kind;
ASTJoin::Strictness strictness;
@ -148,8 +194,10 @@ private:
*/
BlocksList blocks;
MapsAny maps_any;
MapsAll maps_all;
MapsAny maps_any; /// Для ANY LEFT|INNER JOIN
MapsAll maps_all; /// Для ALL LEFT|INNER JOIN
MapsAnyFull maps_any_full; /// Для ANY RIGHT|FULL JOIN
MapsAllFull maps_all_full; /// Для ALL RIGHT|FULL JOIN
/// Дополнительные данные - строки, а также продолжения односвязных списков строк.
Arena pool;
@ -176,6 +224,8 @@ private:
size_t max_bytes;
OverflowMode overflow_mode;
Block totals;
/** Защищает работу с состоянием в функциях insertFromBlock и joinBlock.
* Эти функции могут вызываться одновременно из разных потоков только при использовании StorageJoin,
* и StorageJoin вызывает только эти две функции.

View File

@ -22,46 +22,49 @@ namespace DB
* Также реализует ограничение на их количество.
*/
/// Запрос и данные о его выполнении.
struct ProcessListElement
{
String query;
String user;
String query_id;
Poco::Net::IPAddress ip_address;
Stopwatch watch;
Progress progress;
MemoryTracker memory_tracker;
bool is_cancelled = false;
ProcessListElement(const String & query_, const String & user_,
const String & query_id_, const Poco::Net::IPAddress & ip_address_,
size_t max_memory_usage)
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage)
{
current_memory_tracker = &memory_tracker;
}
~ProcessListElement()
{
current_memory_tracker = nullptr;
}
bool update(const Progress & value)
{
progress.incrementPiecewiseAtomically(value);
return !is_cancelled;
}
};
class ProcessList
{
friend class Entry;
public:
/// Запрос и данные о его выполнении.
struct Element
{
String query;
String user;
String query_id;
Poco::Net::IPAddress ip_address;
Stopwatch watch;
Progress progress;
MemoryTracker memory_tracker;
bool is_cancelled = false;
Element(const String & query_, const String & user_,
const String & query_id_, const Poco::Net::IPAddress & ip_address_,
size_t max_memory_usage)
: query(query_), user(user_), query_id(query_id_), ip_address(ip_address_), memory_tracker(max_memory_usage)
{
current_memory_tracker = &memory_tracker;
}
~Element()
{
current_memory_tracker = nullptr;
}
bool update(const Progress & value)
{
progress.incrementPiecewiseAtomically(value);
return !is_cancelled;
}
};
using Element = ProcessListElement;
/// list, чтобы итераторы не инвалидировались. NOTE: можно заменить на cyclic buffer, но почти незачем.
typedef std::list<Element> Containter;

View File

@ -2,6 +2,7 @@
#include <string.h>
#include <unordered_map>
#include <memory>
#include <Poco/Timespan.h>
@ -129,7 +130,7 @@ public:
{
return cont.empty();
}
void initFromConfig(const String & config_elem, Poco::Util::AbstractConfiguration & config);
/// Обновляет максимальные значения значениями из quota.
@ -154,7 +155,7 @@ public:
}
};
typedef Poco::SharedPtr<QuotaForIntervals> QuotaForIntervalsPtr;
typedef std::shared_ptr<QuotaForIntervals> QuotaForIntervalsPtr;
/// Ключ квоты -> квоты за интервалы. Если квота не допускает ключей, то накопленные значения хранятся по ключу 0.

View File

@ -37,6 +37,8 @@ struct Settings
M(SettingUInt64, max_insert_block_size, DEFAULT_INSERT_BLOCK_SIZE) \
/** Максимальное количество потоков выполнения запроса. По-умолчанию - определять автоматически. */ \
M(SettingMaxThreads, max_threads, 0) \
/** Максимальный размер буфера для чтения из файловой системы. */ \
M(SettingUInt64, max_read_buffer_size, DBMS_DEFAULT_BUFFER_SIZE) \
/** Максимальное количество соединений при распределённой обработке одного запроса (должно быть больше, чем max_threads). */ \
M(SettingUInt64, max_distributed_connections, DEFAULT_MAX_DISTRIBUTED_CONNECTIONS) \
/** Какую часть запроса можно прочитать в оперативку для парсинга (оставшиеся данные для INSERT, если есть, считываются позже) */ \
@ -110,11 +112,14 @@ struct Settings
/** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \
M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \
\
/** Минимальное количество байтов для операций ввода/ввывода минуя кэш страниц */ \
M(SettingUInt64, min_bytes_to_use_direct_io, (20U * 1024U * 1024U * 1024U)) \
/** Минимальное количество байт для операций ввода/ввывода минуя кэш страниц. 0 - отключено. */ \
M(SettingUInt64, min_bytes_to_use_direct_io, 0) \
\
/** Кидать исключение, если есть индекс по дате, и он не используется. */ \
M(SettingBool, force_index_by_date, 0) \
\
/** В запросе INSERT с указанием столбцов, заполнять значения по-умолчанию только для столбцов с явными DEFAULT-ами. */ \
M(SettingBool, strict_insert_defaults, 0) \
/// Всевозможные ограничения на выполнение запроса.
Limits limits;

View File

@ -1,7 +1,6 @@
#pragma once
#include <DB/Core/QueryProcessingStage.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Interpreters/InterpreterQuery.h>

View File

@ -51,7 +51,7 @@ public:
bool detach = false; /// true для DETACH PARTITION.
bool part = false; /// true для ATTACH [UNREPLICATED] PART
bool unreplicated = false; /// true для ATTACH UNREPLICATED ...
bool unreplicated = false; /// true для ATTACH UNREPLICATED, DROP UNREPLICATED ...
/** Для FETCH PARTITION - путь в ZK к шарду, с которого скачивать партицию.
*/

View File

@ -30,7 +30,9 @@ public:
enum Kind
{
Inner, /// Оставить только записи, для которых в "правой" таблице есть соответствующая.
Left /// Если в "правой" таблице нет соответствующих записей, заполнить столбцы значениями "по-умолчанию".
Left, /// Если в "правой" таблице нет соответствующих записей, заполнить столбцы значениями "по-умолчанию".
Right,
Full
};
Locality locality = Local;
@ -54,7 +56,13 @@ public:
writeString("Global", wb);
writeString(strictness == Any ? "Any" : "All", wb);
writeString(kind == Inner ? "Inner" : "Left", wb);
writeString(
kind == Inner ? "Inner"
: (kind == Left ? "Left"
: (kind == Right ? "Right"
: "Full")), wb);
writeString("Join", wb);
}

View File

@ -27,15 +27,15 @@ private:
{
return (c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') || (c >= '0' && c <= '9') || (c == '_');
}
public:
ParserString(const char * s_, bool word_boundary_ = false, bool case_insensitive_ = false)
: s(s_), s_size(strlen(s)), word_boundary(word_boundary_), case_insensitive(case_insensitive_) {}
protected:
const char * getName() const { return s; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
if (static_cast<ssize_t>(s_size) > end - pos || (case_insensitive ? strncasecmp : strncmp)(pos, s, s_size))
return false;
@ -44,14 +44,14 @@ protected:
if (word_boundary && s_size && is_word(s[s_size - 1])
&& pos + s_size != end && is_word(pos[s_size]))
return false;
pos += s_size;
return true;
}
}
};
/** пробельные символы
*/
class ParserWhiteSpace : public IParserBase
@ -61,10 +61,10 @@ public:
protected:
bool allow_newlines;
const char * getName() const { return "white space"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
while (pos < end && (*pos == ' ' || *pos == '\t' || (allow_newlines && *pos == '\n') || *pos == '\r' || *pos == '\f'))
@ -80,7 +80,7 @@ class ParserCStyleComment : public IParserBase
protected:
const char * getName() const { return "C-style comment"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
if (end - pos >= 4 && pos[0] == '/' && pos[1] == '*')
{
@ -110,7 +110,7 @@ class ParserSQLStyleComment : public IParserBase
protected:
const char * getName() const { return "SQL-style comment"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
if (end - pos >= 2 && pos[0] == '-' && pos[1] == '-')
{
@ -135,13 +135,13 @@ class ParserComment : public IParserBase
protected:
const char * getName() const { return "comment"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserCStyleComment p1;
ParserSQLStyleComment p2;
return p1.ignore(pos, end, expected)
|| p2.ignore(pos, end, expected);
return p1.ignore(pos, end, max_parsed_pos, expected)
|| p2.ignore(pos, end, max_parsed_pos, expected);
}
};
@ -150,19 +150,19 @@ class ParserWhiteSpaceOrComments : public IParserBase
{
public:
ParserWhiteSpaceOrComments(bool allow_newlines_outside_comments_ = true) : allow_newlines_outside_comments(allow_newlines_outside_comments_) {}
protected:
bool allow_newlines_outside_comments;
const char * getName() const { return "white space or comments"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
ParserWhiteSpace p1(allow_newlines_outside_comments);
ParserComment p2;
bool res = false;
while (p1.ignore(pos, end, expected) || p2.ignore(pos, end, expected))
while (p1.ignore(pos, end, max_parsed_pos, expected) || p2.ignore(pos, end, max_parsed_pos, expected))
res = true;
return res;
}

View File

@ -11,7 +11,7 @@ class ParserArray : public IParserBase
{
protected:
const char * getName() const { return "array"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -23,7 +23,7 @@ class ParserParenthesisExpression : public IParserBase
{
protected:
const char * getName() const { return "parenthesized expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -33,7 +33,7 @@ class ParserSubquery : public IParserBase
{
protected:
const char * getName() const { return "SELECT subquery"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -43,7 +43,7 @@ class ParserIdentifier : public IParserBase
{
protected:
const char * getName() const { return "identifier"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -53,7 +53,7 @@ class ParserCompoundIdentifier : public IParserBase
{
protected:
const char * getName() const { return "compound identifier"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -66,7 +66,7 @@ class ParserFunction : public IParserBase
{
protected:
const char * getName() const { return "function"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -76,7 +76,7 @@ class ParserNull : public IParserBase
{
protected:
const char * getName() const { return "NULL"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -86,7 +86,7 @@ class ParserNumber : public IParserBase
{
protected:
const char * getName() const { return "number"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -96,7 +96,7 @@ class ParserStringLiteral : public IParserBase
{
protected:
const char * getName() const { return "string literal"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -106,7 +106,7 @@ class ParserLiteral : public IParserBase
{
protected:
const char * getName() const { return "literal"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -116,7 +116,7 @@ class ParserAlias : public IParserBase
{
protected:
const char * getName() const { return "alias"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -126,7 +126,7 @@ class ParserExpressionElement : public IParserBase
{
protected:
const char * getName() const { return "element of expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -138,9 +138,9 @@ public:
ParserWithOptionalAlias(ParserPtr && elem_parser_) : elem_parser(std::move(elem_parser_)) {}
protected:
ParserPtr elem_parser;
const char * getName() const { return "element of expression with optional alias"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -151,7 +151,7 @@ class ParserOrderByElement : public IParserBase
{
protected:
const char * getName() const { return "element of ORDER BY expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};

View File

@ -25,7 +25,7 @@ public:
}
protected:
const char * getName() const { return "list of elements"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
private:
ParserPtr elem_parser;
ParserPtr separator_parser;
@ -57,11 +57,11 @@ public:
remaining_elem_parser(std::move(remaining_elem_parser_))
{
}
protected:
const char * getName() const { return "list, delimited by binary operators"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -84,7 +84,7 @@ public:
protected:
const char * getName() const { return "list, delimited by operator of variable arity"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -104,10 +104,10 @@ public:
: operators(operators_), elem_parser(std::move(elem_parser_))
{
}
protected:
const char * getName() const { return "expression with prefix unary operator"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -118,8 +118,8 @@ private:
protected:
const char * getName() const { return "access expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -128,11 +128,11 @@ class ParserUnaryMinusExpression : public IParserBase
private:
static const char * operators[];
ParserPrefixUnaryOperatorExpression operator_parser {operators, ParserPtr(new ParserAccessExpression)};
protected:
const char * getName() const { return "unary minus expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -144,10 +144,10 @@ private:
protected:
const char * getName() const { return "multiplicative expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return operator_parser.parse(pos, end, node, expected);
return operator_parser.parse(pos, end, node, max_parsed_pos, expected);
}
};
@ -157,13 +157,13 @@ class ParserAdditiveExpression : public IParserBase
private:
static const char * operators[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, ParserPtr(new ParserMultiplicativeExpression)};
protected:
const char * getName() const { return "additive expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return operator_parser.parse(pos, end, node, expected);
return operator_parser.parse(pos, end, node, max_parsed_pos, expected);
}
};
@ -173,13 +173,13 @@ class ParserComparisonExpression : public IParserBase
private:
static const char * operators[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, ParserPtr(new ParserAdditiveExpression)};
protected:
const char * getName() const { return "comparison expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return operator_parser.parse(pos, end, node, expected);
return operator_parser.parse(pos, end, node, max_parsed_pos, expected);
}
};
@ -192,10 +192,10 @@ private:
protected:
const char * getName() const { return "logical-NOT expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return operator_parser.parse(pos, end, node, expected);
return operator_parser.parse(pos, end, node, max_parsed_pos, expected);
}
};
@ -209,13 +209,13 @@ public:
: operator_parser("AND", "and", ParserPtr(new ParserLogicalNotExpression))
{
}
protected:
const char * getName() const { return "logical-AND expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return operator_parser.parse(pos, end, node, expected);
return operator_parser.parse(pos, end, node, max_parsed_pos, expected);
}
};
@ -229,13 +229,13 @@ public:
: operator_parser("OR", "or", ParserPtr(new ParserLogicalAndExpression))
{
}
protected:
const char * getName() const { return "logical-OR expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return operator_parser.parse(pos, end, node, expected);
return operator_parser.parse(pos, end, node, max_parsed_pos, expected);
}
};
@ -251,7 +251,7 @@ private:
protected:
const char * getName() const { return "expression with ternary operator"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -259,11 +259,11 @@ class ParserLambdaExpression : public IParserBase
{
private:
ParserTernaryOperatorExpression elem_parser;
protected:
const char * getName() const { return "lambda expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -275,10 +275,10 @@ protected:
ParserPtr impl;
const char * getName() const { return "expression with optional alias"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
return impl->parse(pos, end, node, expected);
return impl->parse(pos, end, node, max_parsed_pos, expected);
}
};
@ -288,7 +288,7 @@ class ParserExpressionList : public IParserBase
{
protected:
const char * getName() const { return "list of expressions"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -298,7 +298,7 @@ private:
ParserExpressionList nested_parser;
protected:
const char * getName() const { return "not empty list of expressions"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -306,7 +306,7 @@ class ParserOrderByExpressionList : public IParserBase
{
protected:
const char * getName() const { return "order by expression"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};

View File

@ -13,7 +13,7 @@ namespace DB
using Poco::SharedPtr;
typedef const char * Expected;
using Expected = const char *;
/** Интерфейс для классов-парсеров
@ -21,7 +21,7 @@ typedef const char * Expected;
class IParser
{
public:
typedef const char * Pos;
using Pos = const char *;
/** Получить текст о том, что парсит этот парсер. */
virtual const char * getName() const = 0;
@ -34,27 +34,28 @@ public:
* или что парсит этот парсер, если парсинг был успешным.
* Строка, в которую входит диапазон [begin, end) может быть не 0-terminated.
*/
virtual bool parse(Pos & pos, Pos end, ASTPtr & node, Expected & expected) = 0;
virtual bool parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) = 0;
bool ignore(Pos & pos, Pos end, Expected & expected)
bool ignore(Pos & pos, Pos end, Pos & max_parsed_pos, Expected & expected)
{
ASTPtr ignore_node;
return parse(pos, end, ignore_node, expected);
return parse(pos, end, ignore_node, max_parsed_pos, expected);
}
bool ignore(Pos & pos, Pos end)
{
Pos max_parsed_pos = pos;
Expected expected;
return ignore(pos, end, expected);
return ignore(pos, end, max_parsed_pos, expected);
}
/** То же самое, но не двигать позицию и не записывать результат в node.
*/
bool check(Pos & pos, Pos end, Expected & expected)
bool check(Pos & pos, Pos end, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
ASTPtr node;
if (!parse(pos, end, node, expected))
if (!parse(pos, end, node, max_parsed_pos, expected))
{
pos = begin;
return false;
@ -66,62 +67,6 @@ public:
virtual ~IParser() {}
};
typedef std::unique_ptr<IParser> ParserPtr;
/** Из позиции в (возможно многострочном) запросе получить номер строки и номер столбца в строке.
* Используется в сообщении об ошибках.
*/
inline std::pair<size_t, size_t> getLineAndCol(IParser::Pos begin, IParser::Pos pos)
{
size_t line = 0;
IParser::Pos nl;
while (nullptr != (nl = reinterpret_cast<IParser::Pos>(memchr(begin, '\n', pos - begin))))
{
++line;
begin = nl + 1;
}
/// Нумеруются с единицы.
return { line + 1, pos - begin + 1 };
}
/** Получить сообщение о синтаксической ошибке.
*/
inline std::string getSyntaxErrorMessage(
bool parse_res, /// false, если не удалось распарсить; true, если удалось, но не до конца строки или точки с запятой.
IParser::Pos begin,
IParser::Pos end,
IParser::Pos pos,
Expected expected,
const std::string & description = "")
{
std::stringstream message;
message << "Syntax error";
if (!description.empty())
message << " (" << description << ")";
message << ": failed at position " << (pos - begin + 1);
/// Если запрос многострочный.
IParser::Pos nl = reinterpret_cast<IParser::Pos>(memchr(begin, '\n', end - begin));
if (nullptr != nl && nl + 1 != end)
{
size_t line = 0;
size_t col = 0;
std::tie(line, col) = getLineAndCol(begin, pos);
message << " (line " << line << ", col " << col << ")";
}
message << ": " << std::string(pos, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, end - pos))
<< ", expected " << (parse_res ? "end of query" : expected) << ".";
return message.str();
}
using ParserPtr = std::unique_ptr<IParser>;
}

View File

@ -1,12 +1,7 @@
#pragma once
#include <list>
#include <Poco/SharedPtr.h>
#include <DB/Core/Types.h>
#include <DB/Parsers/IParser.h>
#include <iostream>
namespace DB
{
@ -16,21 +11,10 @@ namespace DB
class IParserBase : public IParser
{
public:
bool parse(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
{
expected = getName();
bool res = parseImpl(pos, end, node, expected);
bool parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
if (!res)
node = nullptr;
if (pos > end)
throw Exception("Logical error: pos > end.", ErrorCodes::LOGICAL_ERROR);
return res;
}
protected:
virtual bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected) = 0;
virtual bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) = 0;
};
}

View File

@ -18,7 +18,7 @@ class ParserAlterQuery : public IParserBase
{
protected:
const char * getName() const { return "ALTER query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -11,7 +11,7 @@ class ParserCheckQuery : public IParserBase
{
protected:
const char * getName() const { return "ALTER query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -19,7 +19,7 @@ class ParserNestedTable : public IParserBase
{
protected:
const char * getName() const { return "nested table"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -33,7 +33,7 @@ class ParserIdentifierWithParameters : public IParserBase
{
protected:
const char * getName() const { return "identifier with parameters"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -44,7 +44,7 @@ class ParserIdentifierWithOptionalParameters : public IParserBase
{
protected:
const char * getName() const { return "identifier with optional parameters"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -53,7 +53,7 @@ class IParserNameTypePair : public IParserBase
{
protected:
const char * getName() const { return "name and type pair"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
/** Имя и тип через пробел. Например, URL String. */
@ -62,7 +62,7 @@ typedef IParserNameTypePair<ParserIdentifier> ParserNameTypePair;
typedef IParserNameTypePair<ParserCompoundIdentifier> ParserCompoundNameTypePair;
template <class NameParser>
bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
NameParser name_parser;
ParserIdentifierWithOptionalParameters type_parser;
@ -71,9 +71,9 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & nod
Pos begin = pos;
ASTPtr name, type;
if (name_parser.parse(pos, end, name, expected)
&& ws_parser.ignore(pos, end, expected)
&& type_parser.parse(pos, end, type, expected))
if (name_parser.parse(pos, end, name, max_parsed_pos, expected)
&& ws_parser.ignore(pos, end, max_parsed_pos, expected)
&& type_parser.parse(pos, end, type, max_parsed_pos, expected))
{
ASTNameTypePair * name_type_pair = new ASTNameTypePair(StringRange(begin, pos));
node = name_type_pair;
@ -83,7 +83,6 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & nod
return true;
}
pos = begin;
return false;
}
@ -92,7 +91,7 @@ class ParserNameTypePairList : public IParserBase
{
protected:
const char * getName() const { return "name and type pair list"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -101,14 +100,14 @@ class IParserColumnDeclaration : public IParserBase
{
protected:
const char * getName() const { return "column declaration"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
typedef IParserColumnDeclaration<ParserIdentifier> ParserColumnDeclaration;
typedef IParserColumnDeclaration<ParserCompoundIdentifier> ParserCompoundColumnDeclaration;
template <class NameParser>
bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
NameParser name_parser;
ParserIdentifierWithOptionalParameters type_parser;
@ -119,28 +118,25 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
ParserTernaryOperatorExpression expr_parser;
const auto begin = pos;
const auto reset_pos_and_return = [&pos, begin] {
pos = begin;
return false;
};
/// mandatory column name
ASTPtr name;
if (!name_parser.parse(pos, end, name, expected))
if (!name_parser.parse(pos, end, name, max_parsed_pos, expected))
return false;
ws.ignore(pos, end, expected);
ws.ignore(pos, end, max_parsed_pos, expected);
/** column name should be followed by type name if it
* is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS} */
* is not immediately followed by {DEFAULT, MATERIALIZED, ALIAS}
*/
ASTPtr type;
const auto fallback_pos = pos;
if (!s_default.check(pos, end, expected) &&
!s_materialized.check(pos, end, expected) &&
!s_alias.check(pos, end, expected))
if (!s_default.check(pos, end, expected, max_parsed_pos) &&
!s_materialized.check(pos, end, expected, max_parsed_pos) &&
!s_alias.check(pos, end, expected, max_parsed_pos))
{
if (type_parser.parse(pos, end, type, expected))
ws.ignore(pos, end, expected);
if (type_parser.parse(pos, end, type, max_parsed_pos, expected))
ws.ignore(pos, end, max_parsed_pos, expected);
}
else
pos = fallback_pos;
@ -149,20 +145,20 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
String default_specifier;
ASTPtr default_expression;
const auto pos_before_specifier = pos;
if (s_default.ignore(pos, end, expected) ||
s_materialized.ignore(pos, end, expected) ||
s_alias.ignore(pos, end, expected))
if (s_default.ignore(pos, end, max_parsed_pos, expected) ||
s_materialized.ignore(pos, end, max_parsed_pos, expected) ||
s_alias.ignore(pos, end, max_parsed_pos, expected))
{
default_specifier = Poco::toUpper(std::string{pos_before_specifier, pos});
/// should be followed by an expression
ws.ignore(pos, end, expected);
ws.ignore(pos, end, max_parsed_pos, expected);
if (!expr_parser.parse(pos, end, default_expression, expected))
return reset_pos_and_return();
if (!expr_parser.parse(pos, end, default_expression, max_parsed_pos, expected))
return false;
}
else if (!type)
return reset_pos_and_return(); /// reject sole column name without type
return false; /// reject sole column name without type
const auto column_declaration = new ASTColumnDeclaration{StringRange{begin, pos}};
node = column_declaration;
@ -187,7 +183,7 @@ class ParserColumnDeclarationList : public IParserBase
{
protected:
const char * getName() const { return "column declaration list"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -196,7 +192,7 @@ class ParserEngine : public IParserBase
{
protected:
const char * getName() const { return "ENGINE"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
@ -224,7 +220,7 @@ class ParserCreateQuery : public IParserBase
{
protected:
const char * getName() const { return "CREATE TABLE or ATTACH TABLE query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -17,7 +17,7 @@ class ParserDropQuery : public IParserBase
{
protected:
const char * getName() const { return "DROP query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -26,7 +26,7 @@ class ParserInsertQuery : public IParserBase
{
protected:
const char * getName() const { return "INSERT query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -11,7 +11,7 @@ class ParserJoin : public IParserBase
{
protected:
const char * getName() const { return "JOIN"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -13,7 +13,7 @@ class ParserOptimizeQuery : public IParserBase
{
protected:
const char * getName() const { return "OPTIMIZE query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -11,7 +11,7 @@ class ParserQuery : public IParserBase
{
protected:
const char * getName() const { return "Query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -15,7 +15,7 @@ class ParserRenameQuery : public IParserBase
{
protected:
const char * getName() const { return "RENAME query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -11,7 +11,7 @@ class ParserSelectQuery : public IParserBase
{
protected:
const char * getName() const { return "SELECT query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -14,7 +14,7 @@ class ParserSetQuery : public IParserBase
{
protected:
const char * getName() const { return "SET query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -17,7 +17,7 @@ class ParserShowProcesslistQuery : public IParserBase
protected:
const char * getName() const { return "SHOW PROCESSLIST query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
@ -30,23 +30,23 @@ protected:
ws.ignore(pos, end);
if (!s_show.ignore(pos, end, expected))
if (!s_show.ignore(pos, end, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
if (!s_processlist.ignore(pos, end, expected))
if (!s_processlist.ignore(pos, end, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
if (s_format.ignore(pos, end, expected))
if (s_format.ignore(pos, end, max_parsed_pos, expected))
{
ws.ignore(pos, end);
ParserIdentifier format_p;
if (!format_p.parse(pos, end, format, expected))
if (!format_p.parse(pos, end, format, max_parsed_pos, expected))
return false;
typeid_cast<ASTIdentifier &>(*format).kind = ASTIdentifier::Format;

View File

@ -15,7 +15,7 @@ class ParserShowTablesQuery : public IParserBase
{
protected:
const char * getName() const { return "SHOW TABLES|DATABASES query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -13,7 +13,7 @@ class ParserTablePropertiesQuery : public IParserBase
{
protected:
const char * getName() const { return "EXISTS, SHOW CREATE or DESCRIBE query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected);
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
};
}

View File

@ -16,7 +16,7 @@ class ParserUseQuery : public IParserBase
{
protected:
const char * getName() const { return "USE query"; }
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Expected & expected)
bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
@ -28,12 +28,12 @@ protected:
ws.ignore(pos, end);
if (!s_use.ignore(pos, end, expected))
if (!s_use.ignore(pos, end, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);
if (!name_p.parse(pos, end, database, expected))
if (!name_p.parse(pos, end, database, max_parsed_pos, expected))
return false;
ws.ignore(pos, end);

View File

@ -0,0 +1,33 @@
#pragma once
#include <DB/Parsers/IParser.h>
namespace DB
{
/// Распарсить запрос или записать сообщение об ошибке в out_error_message.
ASTPtr tryParseQuery(
IParser & parser,
IParser::Pos & pos, /// Сдвигается до конца распарсенного фрагмента.
IParser::Pos end,
std::string & out_error_message,
bool hilite,
const std::string & description);
/// Распарсить запрос или кинуть исключение с сообщением об ошибке.
ASTPtr parseQueryAndMovePosition(
IParser & parser,
IParser::Pos & pos, /// Сдвигается до конца распарсенного фрагмента.
IParser::Pos end,
const std::string & description);
ASTPtr parseQuery(
IParser & parser,
IParser::Pos begin,
IParser::Pos end,
const std::string & description);
}

View File

@ -1,21 +1,22 @@
#pragma once
#include <DB/Parsers/IAST.h>
#include <DB/Parsers/queryToString.h>
#include <DB/Parsers/ExpressionListParsers.h>
#include <DB/IO/WriteBufferFromString.h>
#include <unordered_map>
namespace DB
{
enum struct ColumnDefaultType
{
Default,
Materialized,
Alias
};
enum class ColumnDefaultType
{
Default,
Materialized,
Alias
};
}
namespace std
{
template<> struct hash<DB::ColumnDefaultType>
@ -27,152 +28,51 @@ namespace std
};
}
namespace DB
{
inline ColumnDefaultType columnDefaultTypeFromString(const String & str)
{
static const std::unordered_map<String, ColumnDefaultType> map{
{ "DEFAULT", ColumnDefaultType::Default },
{ "MATERIALIZED", ColumnDefaultType::Materialized },
{ "ALIAS", ColumnDefaultType::Alias }
};
const auto it = map.find(str);
return it != std::end(map) ? it->second : throw Exception{"Unknown column default specifier: " + str};
}
inline String toString(const ColumnDefaultType type)
{
static const std::unordered_map<ColumnDefaultType, String> map{
{ ColumnDefaultType::Default, "DEFAULT" },
{ ColumnDefaultType::Materialized, "MATERIALIZED" },
{ ColumnDefaultType::Alias, "ALIAS" }
};
const auto it = map.find(type);
return it != std::end(map) ? it->second : throw Exception{"Invalid ColumnDefaultType"};
}
struct ColumnDefault
{
ColumnDefaultType type;
ASTPtr expression;
inline ColumnDefaultType columnDefaultTypeFromString(const String & str)
{
static const std::unordered_map<String, ColumnDefaultType> map{
{ "DEFAULT", ColumnDefaultType::Default },
{ "MATERIALIZED", ColumnDefaultType::Materialized },
{ "ALIAS", ColumnDefaultType::Alias }
};
inline bool operator==(const ColumnDefault & lhs, const ColumnDefault & rhs)
{
return lhs.type == rhs.type && queryToString(lhs.expression) == queryToString(rhs.expression);
}
using ColumnDefaults = std::unordered_map<String, ColumnDefault>;
template <bool store>
struct ColumnsDescription
{
template <typename T> using by_value_or_cref = typename std::conditional<store, T, const T &>::type;
by_value_or_cref<NamesAndTypesList> columns;
by_value_or_cref<NamesAndTypesList> materialized;
by_value_or_cref<NamesAndTypesList> alias;
by_value_or_cref<ColumnDefaults> defaults;
String toString() const
{
String s;
WriteBufferFromString buf{s};
writeString("columns format version: 1\n", buf);
writeText(columns.size() + materialized.size() + alias.size(), buf);
writeString(" columns:\n", buf);
const auto write_columns = [this, &buf] (const NamesAndTypesList & columns) {
for (const auto & column : columns)
{
const auto it = defaults.find(column.name);
writeBackQuotedString(column.name, buf);
writeChar(' ', buf);
writeString(column.type->getName(), buf);
if (it == std::end(defaults))
{
writeChar('\n', buf);
continue;
}
else
writeChar('\t', buf);
writeString(DB::toString(it->second.type), buf);
writeChar('\t', buf);
writeString(queryToString(it->second.expression), buf);
writeChar('\n', buf);
}
};
write_columns(columns);
write_columns(materialized);
write_columns(alias);
return s;
}
static ColumnsDescription parse(const String & str, const DataTypeFactory & data_type_factory)
{
ReadBufferFromString buf{str};
assertString("columns format version: 1\n", buf);
size_t count{};
readText(count, buf);
assertString(" columns:\n", buf);
ParserTernaryOperatorExpression expr_parser;
ColumnsDescription result{};
for (size_t i = 0; i < count; ++i)
{
String column_name;
readBackQuotedString(column_name, buf);
assertString(" ", buf);
String type_name;
readString(type_name, buf);
auto type = data_type_factory.get(type_name);
if (*buf.position() == '\n')
{
assertString("\n", buf);
result.columns.emplace_back(column_name, std::move(type));
continue;
}
assertString("\t", buf);
String default_type_str;
readString(default_type_str, buf);
const auto default_type = columnDefaultTypeFromString(default_type_str);
assertString("\t", buf);
String default_expr_str;
readText(default_expr_str, buf);
assertString("\n", buf);
ASTPtr default_expr;
Expected expected{};
auto begin = default_expr_str.data();
const auto end = begin + default_expr_str.size();
if (!expr_parser.parse(begin, end, default_expr, expected))
throw Exception{"Could not parse default expression", DB::ErrorCodes::CANNOT_PARSE_TEXT};
if (ColumnDefaultType::Default == default_type)
result.columns.emplace_back(column_name, std::move(type));
else if (ColumnDefaultType::Materialized == default_type)
result.materialized.emplace_back(column_name, std::move(type));
else if (ColumnDefaultType::Alias == default_type)
result.alias.emplace_back(column_name, std::move(type));
result.defaults.emplace(column_name, ColumnDefault{default_type, default_expr});
}
assertEOF(buf);
return result;
}
};
const auto it = map.find(str);
return it != std::end(map) ? it->second : throw Exception{"Unknown column default specifier: " + str};
}
inline String toString(const ColumnDefaultType type)
{
static const std::unordered_map<ColumnDefaultType, String> map{
{ ColumnDefaultType::Default, "DEFAULT" },
{ ColumnDefaultType::Materialized, "MATERIALIZED" },
{ ColumnDefaultType::Alias, "ALIAS" }
};
const auto it = map.find(type);
return it != std::end(map) ? it->second : throw Exception{"Invalid ColumnDefaultType"};
}
struct ColumnDefault
{
ColumnDefaultType type;
ASTPtr expression;
};
inline bool operator==(const ColumnDefault & lhs, const ColumnDefault & rhs)
{
return lhs.type == rhs.type && queryToString(lhs.expression) == queryToString(rhs.expression);
}
using ColumnDefaults = std::unordered_map<String, ColumnDefault>;
}

View File

@ -0,0 +1,28 @@
#pragma once
#include <DB/Storages/ColumnDefault.h>
#include <DB/Core/NamesAndTypes.h>
namespace DB
{
template <bool store>
struct ColumnsDescription
{
template <typename T>
using by_value_or_cref = typename std::conditional<store, T, const T &>::type;
by_value_or_cref<NamesAndTypesList> columns;
by_value_or_cref<NamesAndTypesList> materialized;
by_value_or_cref<NamesAndTypesList> alias;
by_value_or_cref<ColumnDefaults> defaults;
String toString() const;
static ColumnsDescription parse(const String & str, const DataTypeFactory & data_type_factory);
};
}

View File

@ -0,0 +1,97 @@
#include <DB/IO/CompressedStream.h>
#include <DB/IO/ReadHelpers.h>
#include <DB/Core/ErrorCodes.h>
#include <statdaemons/Exception.h>
#include <Poco/Util/AbstractConfiguration.h>
namespace DB
{
/** Позволяет выбрать метод сжатия по указанным в конфигурационном файле условиям.
* Конфиг выглядит примерно так:
<compression>
<!-- Набор вариантов. Варианты проверяются подряд. Побеждает последний сработавший вариант. Если ни один не сработал, то используется lz4. -->
<case>
<!-- Условия. Должны сработать одновременно все. Часть условий могут быть не указаны. -->
<min_part_size>10000000000</min_part_size> <!-- Минимальный размер куска в байтах. -->
<min_part_size_ratio>0.01</min_part_size_ratio> <!-- Минимальный размер куска относительно всех данных таблицы. -->
<!-- Какой метод сжатия выбрать. -->
<method>zstd</method>
</case>
<case>
...
</case>
</compression>
*/
class CompressionMethodSelector
{
private:
struct Element
{
size_t min_part_size = 0;
double min_part_size_ratio = 0;
CompressionMethod method = CompressionMethod::LZ4;
void setMethod(const std::string & name)
{
if (name == "lz4")
method = CompressionMethod::LZ4;
else if (name == "zstd")
method = CompressionMethod::ZSTD;
else
throw Exception("Unknown compression method " + name, ErrorCodes::UNKNOWN_COMPRESSION_METHOD);
}
Element(Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
min_part_size = parse<size_t>(config.getString(config_prefix + ".min_part_size", "0"));
min_part_size_ratio = config.getDouble(config_prefix + ".min_part_size_ratio", 0);
setMethod(config.getString(config_prefix + ".method"));
}
bool check(size_t part_size, double part_size_ratio) const
{
return part_size >= min_part_size
&& part_size_ratio >= min_part_size_ratio;
}
};
std::vector<Element> elements;
public:
CompressionMethodSelector() {} /// Всегда возвращает метод по-умолчанию.
CompressionMethodSelector(Poco::Util::AbstractConfiguration & config, const std::string & config_prefix)
{
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(config_prefix, keys);
for (const auto & name : keys)
{
if (0 != strncmp(name.data(), "case", strlen("case")))
throw Exception("Unknown element in config: " + config_prefix + "." + name + ", must be 'case'", ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG);
elements.emplace_back(config, config_prefix + "." + name);
}
}
CompressionMethod choose(size_t part_size, double part_size_ratio) const
{
CompressionMethod res = CompressionMethod::LZ4;
for (const auto & element : elements)
if (element.check(part_size, part_size_ratio))
res = element.method;
return res;
}
};
}

View File

@ -3,10 +3,17 @@
#include <DB/DataStreams/RemoteBlockOutputStream.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <boost/algorithm/string/find_iterator.hpp>
#include <boost/algorithm/string/finder.hpp>
#include <Poco/DirectoryIterator.h>
#include <thread>
#include <mutex>
#include <condition_variable>
namespace DB
{

View File

@ -8,6 +8,7 @@
#include <DB/IO/CompressedWriteBuffer.h>
#include <DB/DataStreams/NativeBlockOutputStream.h>
#include <DB/Interpreters/InterpreterInsertQuery.h>
#include <DB/Interpreters/Cluster.h>
#include <statdaemons/Increment.h>
#include <memory>
@ -16,6 +17,7 @@
#include <iostream>
#include <type_traits>
namespace DB
{

View File

@ -207,7 +207,7 @@ public:
/** Выполнить запрос (DROP|DETACH) PARTITION.
*/
virtual void dropPartition(const Field & partition, bool detach, const Settings & settings)
virtual void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings)
{
throw Exception("Method dropPartition is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
@ -236,7 +236,7 @@ public:
/** Выполнить какую-либо фоновую работу. Например, объединение кусков в таблице типа MergeTree.
* Возвращает - была ли выполнена какая-либо работа.
*/
virtual bool optimize()
virtual bool optimize(const Settings & settings)
{
throw Exception("Method optimize is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}

View File

@ -1,5 +1,7 @@
#pragma once
#include <memory>
#include <DB/Common/LRUCache.h>
#include <DB/Common/ProfileEvents.h>
#include <DB/Common/SipHash.h>
@ -32,7 +34,7 @@ struct MarkInCompressedFile
typedef std::vector<MarkInCompressedFile> MarksInCompressedFile;
/// Оценка количества байт, занимаемых засечками в кеше.
/// Оценка количества байтов, занимаемых засечками в кеше.
struct MarksWeightFunction
{
size_t operator()(const MarksInCompressedFile & marks) const
@ -79,6 +81,6 @@ public:
}
};
typedef Poco::SharedPtr<MarkCache> MarkCachePtr;
typedef std::shared_ptr<MarkCache> MarkCachePtr;
}

View File

@ -10,31 +10,52 @@ namespace DB
{
struct MergeInfo
{
const std::string database;
const std::string table;
const std::string result_part_name;
Stopwatch watch;
Float64 progress{};
std::uint64_t num_parts{};
std::uint64_t total_size_bytes_compressed{};
std::uint64_t total_size_marks{};
std::uint64_t bytes_read_uncompressed{};
std::uint64_t rows_read{};
std::uint64_t bytes_written_uncompressed{};
std::uint64_t rows_written{};
MergeInfo(const std::string & database, const std::string & table, const std::string & result_part_name)
: database{database}, table{table}, result_part_name{result_part_name}
{
}
};
class MergeList;
class MergeListEntry
{
MergeList & list;
using container_t = std::list<MergeInfo>;
container_t::iterator it;
public:
MergeListEntry(const MergeListEntry &) = delete;
MergeListEntry & operator=(const MergeListEntry &) = delete;
MergeListEntry(MergeList & list, const container_t::iterator it) : list(list), it{it} {}
~MergeListEntry();
MergeInfo * operator->() { return &*it; }
};
class MergeList
{
friend class Entry;
struct MergeInfo
{
const std::string database;
const std::string table;
const std::string result_part_name;
Stopwatch watch;
Float64 progress{};
std::uint64_t num_parts{};
std::uint64_t total_size_bytes_compressed{};
std::uint64_t total_size_marks{};
std::uint64_t bytes_read_uncompressed{};
std::uint64_t rows_read{};
std::uint64_t bytes_written_uncompressed{};
std::uint64_t rows_written{};
MergeInfo(const std::string & database, const std::string & table, const std::string & result_part_name)
: database{database}, table{table}, result_part_name{result_part_name}
{
}
};
friend class MergeListEntry;
using container_t = std::list<MergeInfo>;
@ -42,25 +63,7 @@ class MergeList
container_t merges;
public:
class Entry
{
MergeList & list;
container_t::iterator it;
public:
Entry(const Entry &) = delete;
Entry & operator=(const Entry &) = delete;
Entry(MergeList & list, const container_t::iterator it) : list(list), it{it} {}
~Entry()
{
std::lock_guard<std::mutex> lock{list.mutex};
list.merges.erase(it);
}
MergeInfo * operator->() { return &*it; }
};
using Entry = MergeListEntry;
using EntryPtr = std::unique_ptr<Entry>;
template <typename... Args>
@ -78,4 +81,11 @@ public:
};
inline MergeListEntry::~MergeListEntry()
{
std::lock_guard<std::mutex> lock{list.mutex};
list.merges.erase(it);
}
}

View File

@ -17,7 +17,8 @@ public:
size_t block_size_, Names column_names,
MergeTreeData & storage_, const MergeTreeData::DataPartPtr & owned_data_part_,
const MarkRanges & mark_ranges_, bool use_uncompressed_cache_,
ExpressionActionsPtr prewhere_actions_, String prewhere_column_, bool check_columns = true)
ExpressionActionsPtr prewhere_actions_, String prewhere_column_, bool check_columns,
size_t min_bytes_to_use_direct_io_, size_t max_read_buffer_size_)
:
path(path_), block_size(block_size_),
storage(storage_), owned_data_part(owned_data_part_),
@ -26,7 +27,8 @@ public:
use_uncompressed_cache(use_uncompressed_cache_),
prewhere_actions(prewhere_actions_), prewhere_column(prewhere_column_),
log(&Logger::get("MergeTreeBlockInputStream")),
ordered_names{column_names}
ordered_names{column_names},
min_bytes_to_use_direct_io(min_bytes_to_use_direct_io_), max_read_buffer_size(max_read_buffer_size_)
{
std::reverse(remaining_mark_ranges.begin(), remaining_mark_ranges.end());
@ -180,12 +182,19 @@ protected:
if (!reader)
{
UncompressedCache * uncompressed_cache = use_uncompressed_cache ? storage.context.getUncompressedCache() : nullptr;
if (use_uncompressed_cache)
owned_uncompressed_cache = storage.context.getUncompressedCache();
reader.reset(new MergeTreeReader(path, owned_data_part, columns, uncompressed_cache, storage, all_mark_ranges));
owned_mark_cache = storage.context.getMarkCache();
reader.reset(new MergeTreeReader(
path, owned_data_part, columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), storage,
all_mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size));
if (prewhere_actions)
pre_reader.reset(new MergeTreeReader(path, owned_data_part, pre_columns, uncompressed_cache, storage, all_mark_ranges));
pre_reader.reset(new MergeTreeReader(
path, owned_data_part, pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), storage,
all_mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size));
}
if (prewhere_actions)
@ -375,6 +384,12 @@ private:
/// column names in specific order as expected by other stages
Names ordered_names;
size_t min_bytes_to_use_direct_io;
size_t max_read_buffer_size;
UncompressedCachePtr owned_uncompressed_cache;
MarkCachePtr owned_mark_cache;
};
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <DB/Storages/StorageMergeTree.h>
#include <DB/DataStreams/IBlockOutputStream.h>
#include <iomanip>
namespace DB

View File

@ -9,7 +9,9 @@
#include <DB/Storages/MergeTree/ActiveDataPartSet.h>
#include <DB/IO/ReadBufferFromString.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/ReadBufferFromFile.h>
#include <DB/Common/escapeForFileName.h>
#include <DB/Common/SipHash.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <Poco/RWLock.h>
@ -258,6 +260,8 @@ public:
/// Описание столбцов.
NamesAndTypesList columns;
using ColumnToSize = std::map<std::string, size_t>;
/** Блокируется на запись при изменении columns, checksums или любых файлов куска.
* Блокируется на чтение при чтении columns, checksums или любых файлов куска.
*/
@ -299,7 +303,7 @@ public:
}
}
/// Вычисляем сумарный размер всей директории со всеми файлами
/// Вычисляем суммарный размер всей директории со всеми файлами
static size_t calcTotalSize(const String & from)
{
Poco::File cur(from);
@ -437,6 +441,14 @@ public:
assertEOF(file);
}
void accumulateColumnSizes(ColumnToSize & column_to_size) const
{
Poco::ScopedReadRWLock part_lock(columns_lock);
for (const NameAndTypePair & column : *storage.columns)
if (Poco::File(storage.full_path + name + "/" + escapeForFileName(column.name) + ".bin").exists())
column_to_size[column.name] += Poco::File(storage.full_path + name + "/" + escapeForFileName(column.name) + ".bin").getSize();
}
void loadColumns(bool require)
{
String path = storage.full_path + name + "/columns.txt";
@ -705,6 +717,10 @@ public:
DataPartsVector getDataPartsVector();
DataParts getAllDataParts();
/** Размер активной части в количестве байт.
*/
size_t getTotalActiveSizeInBytes();
/** Максимальное количество кусков в одном месяце.
*/
size_t getMaxPartsCountForMonth();

View File

@ -7,6 +7,9 @@
namespace DB
{
class MergeListEntry;
/** Умеет выбирать куски для слияния и сливать их.
*/
class MergeTreeDataMerger
@ -41,8 +44,9 @@ public:
* приблизительно пропорционально количеству уже выписанных данных.
*/
MergeTreeData::DataPartPtr mergeParts(
const MergeTreeData::DataPartsVector & parts, const String & merged_name, MergeList::Entry & merge_entry,
MergeTreeData::Transaction * out_transaction = nullptr, DiskSpaceMonitor::Reservation * disk_reservation = nullptr);
const MergeTreeData::DataPartsVector & parts, const String & merged_name, MergeListEntry & merge_entry,
size_t aio_threshold, MergeTreeData::Transaction * out_transaction = nullptr,
DiskSpaceMonitor::Reservation * disk_reservation = nullptr);
/// Примерное количество места на диске, нужное для мерджа. С запасом.
size_t estimateDiskSpaceForMerge(const MergeTreeData::DataPartsVector & parts);

View File

@ -1,7 +1,8 @@
#pragma once
#include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/Storages/MergeTree/MergeTreeBlockInputStream.h>
#include <DB/Storages/MergeTree/MergeTreeReader.h>
namespace DB
{

View File

@ -1,5 +1,6 @@
#pragma once
#include <DB/Storages/MarkCache.h>
#include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/DataTypes/IDataType.h>
#include <DB/DataTypes/DataTypeNested.h>
@ -40,9 +41,13 @@ class MergeTreeReader
public:
MergeTreeReader(const String & path_, /// Путь к куску
const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_,
bool use_uncompressed_cache_, MergeTreeData & storage_, const MarkRanges & all_mark_ranges)
UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_,
MergeTreeData & storage_, const MarkRanges & all_mark_ranges,
size_t aio_threshold_, size_t max_read_buffer_size_)
: path(path_), data_part(data_part), part_name(data_part->name), columns(columns_),
use_uncompressed_cache(use_uncompressed_cache_), storage(storage_), all_mark_ranges(all_mark_ranges)
uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_),
storage(storage_), all_mark_ranges(all_mark_ranges),
aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_)
{
try
{
@ -223,8 +228,9 @@ private:
/// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины.
double avg_value_size_hint = 0;
Stream(const String & path_prefix, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const MarkRanges & all_mark_ranges)
: path_prefix(path_prefix)
Stream(const String & path_prefix_, UncompressedCache * uncompressed_cache, MarkCache * mark_cache, const MarkRanges & all_mark_ranges,
size_t aio_threshold, size_t max_read_buffer_size)
: path_prefix(path_prefix_)
{
loadMarks(mark_cache);
size_t max_mark_range = 0;
@ -245,7 +251,7 @@ private:
if (right >= (*marks).size() || (right + 1 == (*marks).size() &&
(*marks)[right].offset_in_compressed_file == (*marks)[all_mark_ranges[i].end].offset_in_compressed_file))
{
max_mark_range = DBMS_DEFAULT_BUFFER_SIZE;
max_mark_range = max_read_buffer_size;
break;
}
@ -253,16 +259,36 @@ private:
(*marks)[right].offset_in_compressed_file - (*marks)[all_mark_ranges[i].begin].offset_in_compressed_file);
}
size_t buffer_size = DBMS_DEFAULT_BUFFER_SIZE < max_mark_range ? DBMS_DEFAULT_BUFFER_SIZE : max_mark_range;
size_t buffer_size = std::min(max_read_buffer_size, max_mark_range);
size_t estimated_size = 0;
if (aio_threshold > 0)
{
for (const auto & mark_range : all_mark_ranges)
{
size_t offset_begin = (*marks)[mark_range.begin].offset_in_compressed_file;
size_t offset_end;
if (mark_range.end < (*marks).size())
offset_end = (*marks)[mark_range.end].offset_in_compressed_file;
else
offset_end = Poco::File(path_prefix + ".bin").getSize();
if (offset_end > 0)
estimated_size += offset_end - offset_begin;
}
}
if (uncompressed_cache)
{
cached_buffer = new CachedCompressedReadBuffer(path_prefix + ".bin", uncompressed_cache, buffer_size);
cached_buffer = new CachedCompressedReadBuffer(path_prefix + ".bin", uncompressed_cache,
estimated_size, aio_threshold, buffer_size);
data_buffer = &*cached_buffer;
}
else
{
non_cached_buffer = new CompressedReadBufferFromFile(path_prefix + ".bin", buffer_size);
non_cached_buffer = new CompressedReadBufferFromFile(path_prefix + ".bin", estimated_size,
aio_threshold, buffer_size);
data_buffer = &*non_cached_buffer;
}
}
@ -312,7 +338,7 @@ private:
{
/// Более хорошая диагностика.
if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND)
throw Exception(e.message() + " (while seeking to mark " + Poco::NumberFormatter::format(index)
throw Exception(e.message() + " (while seeking to mark " + toString(index)
+ " of column " + path_prefix + "; offsets are: "
+ toString(mark.offset_in_compressed_file) + " "
+ toString(mark.offset_in_decompressed_block) + ")", e.code());
@ -333,10 +359,13 @@ private:
NamesAndTypesList columns;
const NameAndTypePair * added_minimum_size_column = nullptr;
bool use_uncompressed_cache;
UncompressedCache * uncompressed_cache;
MarkCache * mark_cache;
MergeTreeData & storage;
const MarkRanges & all_mark_ranges;
size_t aio_threshold;
size_t max_read_buffer_size;
void addStream(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, size_t level = 0)
{
@ -348,9 +377,6 @@ private:
if (!Poco::File(path + escaped_column_name + ".bin").exists())
return;
UncompressedCache * uncompressed_cache = use_uncompressed_cache ? storage.context.getUncompressedCache() : NULL;
MarkCache * mark_cache = storage.context.getMarkCache();
/// Для массивов используются отдельные потоки для размеров.
if (const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(&type))
{
@ -361,12 +387,13 @@ private:
if (!streams.count(size_name))
streams.emplace(size_name, std::unique_ptr<Stream>(new Stream(
path + escaped_size_name, uncompressed_cache, mark_cache, all_mark_ranges)));
path + escaped_size_name, uncompressed_cache, mark_cache, all_mark_ranges, aio_threshold, max_read_buffer_size)));
addStream(name, *type_arr->getNestedType(), all_mark_ranges, level + 1);
}
else
streams[name].reset(new Stream(path + escaped_column_name, uncompressed_cache, mark_cache, all_mark_ranges));
streams[name].reset(new Stream(
path + escaped_column_name, uncompressed_cache, mark_cache, all_mark_ranges, aio_threshold, max_read_buffer_size));
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <DB/IO/createWriteBufferFromFileBase.h>
#include <DB/IO/WriteBufferFromFile.h>
#include <DB/IO/CompressedWriteBuffer.h>
#include <DB/IO/HashingWriteBuffer.h>
@ -8,6 +9,7 @@
#include <DB/Common/escapeForFileName.h>
#include <DB/DataTypes/DataTypeNested.h>
#include <DB/DataTypes/DataTypeArray.h>
#include <DB/DataStreams/IBlockOutputStream.h>
namespace DB
@ -19,10 +21,12 @@ public:
MergeTreeData & storage_,
size_t min_compress_block_size_,
size_t max_compress_block_size_,
CompressionMethod compression_method_)
CompressionMethod compression_method_,
size_t aio_threshold_)
: storage(storage_),
min_compress_block_size(min_compress_block_size_),
max_compress_block_size(max_compress_block_size_),
aio_threshold(aio_threshold_),
compression_method(compression_method_)
{
}
@ -37,16 +41,18 @@ protected:
const String & data_path,
const std::string & marks_path,
size_t max_compress_block_size,
CompressionMethod compression_method) :
CompressionMethod compression_method,
size_t estimated_size,
size_t aio_threshold) :
escaped_column_name(escaped_column_name_),
plain_file(data_path, max_compress_block_size, O_TRUNC | O_CREAT | O_WRONLY),
plain_hashing(plain_file), compressed_buf(plain_hashing, compression_method), compressed(compressed_buf),
plain_file(createWriteBufferFromFileBase(data_path, estimated_size, aio_threshold, max_compress_block_size)),
plain_hashing(*plain_file), compressed_buf(plain_hashing, compression_method), compressed(compressed_buf),
marks_file(marks_path, 4096, O_TRUNC | O_CREAT | O_WRONLY), marks(marks_file) {}
String escaped_column_name;
/// compressed -> compressed_buf -> plain_hashing -> plain_file
WriteBufferFromFile plain_file;
std::unique_ptr<WriteBufferFromFileBase> plain_file;
HashingWriteBuffer plain_hashing;
CompressedWriteBuffer compressed_buf;
HashingWriteBuffer compressed;
@ -58,13 +64,13 @@ protected:
void finalize()
{
compressed.next();
plain_file.next();
plain_file->next();
marks.next();
}
void sync()
{
plain_file.sync();
plain_file->sync();
marks_file.sync();
}
@ -86,7 +92,7 @@ protected:
using ColumnStreams = std::map<String, std::unique_ptr<ColumnStream>>;
void addStream(const String & path, const String & name, const IDataType & type, size_t level = 0, String filename = "")
void addStream(const String & path, const String & name, const IDataType & type, size_t estimated_size = 0, size_t level = 0, String filename = "")
{
String escaped_column_name;
if (filename.size())
@ -107,9 +113,11 @@ protected:
path + escaped_size_name + ".bin",
path + escaped_size_name + ".mrk",
max_compress_block_size,
compression_method));
compression_method,
estimated_size,
aio_threshold));
addStream(path, name, *type_arr->getNestedType(), level + 1);
addStream(path, name, *type_arr->getNestedType(), estimated_size, level + 1);
}
else
column_streams[name].reset(new ColumnStream(
@ -117,7 +125,9 @@ protected:
path + escaped_column_name + ".bin",
path + escaped_column_name + ".mrk",
max_compress_block_size,
compression_method));
compression_method,
estimated_size,
aio_threshold));
}
@ -215,6 +225,8 @@ protected:
size_t min_compress_block_size;
size_t max_compress_block_size;
size_t aio_threshold;
CompressionMethod compression_method;
};
@ -230,21 +242,42 @@ public:
CompressionMethod compression_method)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method),
storage_.context.getSettings().max_compress_block_size, compression_method,
storage_.context.getSettings().min_bytes_to_use_direct_io),
columns_list(columns_list_), part_path(part_path_)
{
Poco::File(part_path).createDirectories();
if (storage.mode != MergeTreeData::Unsorted)
{
index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
index_stream = new HashingWriteBuffer(*index_file_stream);
}
init();
for (const auto & it : columns_list)
addStream(part_path, it.name, *it.type);
}
MergedBlockOutputStream(
MergeTreeData & storage_,
String part_path_,
const NamesAndTypesList & columns_list_,
CompressionMethod compression_method,
const MergeTreeData::DataPart::ColumnToSize & merged_column_to_size_,
size_t aio_threshold_)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method,
aio_threshold_),
columns_list(columns_list_), part_path(part_path_)
{
init();
for (const auto & it : columns_list)
{
size_t estimated_size = 0;
if (aio_threshold > 0)
{
auto it2 = merged_column_to_size_.find(it.name);
if (it2 != merged_column_to_size_.end())
estimated_size = it2->second;
}
addStream(part_path, it.name, *it.type, estimated_size);
}
}
void write(const Block & block) override
{
size_t rows = block.rows();
@ -346,6 +379,18 @@ public:
return marks_count;
}
private:
void init()
{
Poco::File(part_path).createDirectories();
if (storage.mode != MergeTreeData::Unsorted)
{
index_file_stream = new WriteBufferFromFile(part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY);
index_stream = new HashingWriteBuffer(*index_file_stream);
}
}
private:
NamesAndTypesList columns_list;
String part_path;
@ -366,7 +411,8 @@ public:
MergedColumnOnlyOutputStream(MergeTreeData & storage_, String part_path_, bool sync_, CompressionMethod compression_method)
: IMergedBlockOutputStream(
storage_, storage_.context.getSettings().min_compress_block_size,
storage_.context.getSettings().max_compress_block_size, compression_method),
storage_.context.getSettings().max_compress_block_size, compression_method,
storage_.context.getSettings().min_bytes_to_use_direct_io),
part_path(part_path_), sync(sync_)
{
}
@ -379,7 +425,7 @@ public:
for (size_t i = 0; i < block.columns(); ++i)
{
addStream(part_path, block.getByPosition(i).name,
*block.getByPosition(i).type, 0, block.getByPosition(i).name);
*block.getByPosition(i).type, 0, 0, block.getByPosition(i).name);
}
initialized = true;
}

View File

@ -2,6 +2,7 @@
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <DB/DataStreams/IBlockOutputStream.h>
namespace DB
@ -11,7 +12,7 @@ class ReplicatedMergeTreeBlockOutputStream : public IBlockOutputStream
{
public:
ReplicatedMergeTreeBlockOutputStream(StorageReplicatedMergeTree & storage_, const String & insert_id_)
: storage(storage_), insert_id(insert_id_), block_index(0),
: storage(storage_), insert_id(insert_id_),
log(&Logger::get(storage.data.getLogName() + " (Replicated OutputStream)")) {}
void writePrefix() override
@ -145,7 +146,7 @@ public:
private:
StorageReplicatedMergeTree & storage;
String insert_id;
size_t block_index;
size_t block_index = 0;
Logger * log;

View File

@ -70,7 +70,7 @@ public:
/// Сбрасывает все буферы в подчинённую таблицу.
void shutdown() override;
bool optimize() override;
bool optimize(const Settings & settings) override;
void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override { name = new_table_name; }

View File

@ -4,6 +4,7 @@
#include <DB/Interpreters/Context.h>
#include <DB/Storages/IStorage.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
namespace DB
@ -71,7 +72,7 @@ private:
class MergeTask;
using MergeTaskPtr = std::shared_ptr<MergeTask>;
MergeTaskPtr merge_task;
DB::BackgroundProcessingPool::TaskHandle merge_task_handle;
BackgroundProcessingPool::TaskHandle merge_task_handle;
StorageChunkMerger(
const std::string & this_database_,

View File

@ -5,6 +5,7 @@
#include <DB/Client/ConnectionPoolWithFailover.h>
#include <DB/Interpreters/Settings.h>
#include <DB/Interpreters/Context.h>
#include <DB/Interpreters/ExpressionActions.h>
namespace DB

View File

@ -12,7 +12,7 @@ namespace DB
* а также записаны в файл-бэкап, для восстановления после перезапуска.
* Чтение из таблицы напрямую невозможно - возможно лишь указание в правой части JOIN.
*
* При использовании, JOIN должен быть соответствующего типа (ANY|ALL LEFT|INNER).
* При использовании, JOIN должен быть соответствующего типа (ANY|ALL LEFT|INNER ...).
*/
class StorageJoin : public StorageSetOrJoinBase
{
@ -44,7 +44,7 @@ public:
private:
const Names & key_names;
ASTJoin::Kind kind; /// LEFT | INNER
ASTJoin::Kind kind; /// LEFT | INNER ...
ASTJoin::Strictness strictness; /// ANY | ALL
JoinPtr join;

View File

@ -32,7 +32,7 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
void drop() override;
bool optimize() override;
bool optimize(const Settings & settings) override;
BlockInputStreams read(
const Names & column_names,

View File

@ -5,6 +5,8 @@
#include <DB/Storages/MergeTree/MergeTreeDataWriter.h>
#include <DB/Storages/MergeTree/MergeTreeDataMerger.h>
#include <DB/Storages/MergeTree/DiskSpaceMonitor.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
namespace DB
{
@ -81,12 +83,12 @@ public:
/** Выполнить очередной шаг объединения кусков.
*/
bool optimize() override
bool optimize(const Settings & settings) override
{
return merge(true);
return merge(settings.min_bytes_to_use_direct_io, true);
}
void dropPartition(const Field & partition, bool detach, const Settings & settings) override;
void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) override;
void attachPartition(const Field & partition, bool unreplicated, bool part, const Settings & settings) override;
void freezePartition(const Field & partition, const Settings & settings) override;
@ -188,7 +190,7 @@ private:
* Если aggressive - выбрать куски, не обращая внимание на соотношение размеров и их новизну (для запроса OPTIMIZE).
* Возвращает, получилось ли что-нибудь объединить.
*/
bool merge(bool aggressive = false, BackgroundProcessingPool::Context * context = nullptr);
bool merge(size_t aio_threshold, bool aggressive = false, BackgroundProcessingPool::Context * context = nullptr);
bool mergeTask(BackgroundProcessingPool::Context & context);

View File

@ -10,6 +10,7 @@
#include <DB/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h>
#include <DB/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.h>
#include <DB/Storages/MergeTree/AbandonableLockInZooKeeper.h>
#include <DB/Storages/MergeTree/BackgroundProcessingPool.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <zkutil/ZooKeeper.h>
#include <zkutil/LeaderElection.h>
@ -83,11 +84,11 @@ public:
BlockOutputStreamPtr write(ASTPtr query) override;
bool optimize() override;
bool optimize(const Settings & settings) override;
void alter(const AlterCommands & params, const String & database_name, const String & table_name, Context & context) override;
void dropPartition(const Field & partition, bool detach, const Settings & settings) override;
void dropPartition(const Field & partition, bool detach, bool unreplicated, const Settings & settings) override;
void attachPartition(const Field & partition, bool unreplicated, bool part, const Settings & settings) override;
void fetchPartition(const Field & partition, const String & from, const Settings & settings) override;
void freezePartition(const Field & partition, const Settings & settings) override;
@ -133,6 +134,8 @@ public:
void getStatus(Status & res, bool with_zk_fields = true);
private:
void dropUnreplicatedPartition(const Field & partition, const Settings & settings);
friend class ReplicatedMergeTreeBlockOutputStream;
friend class ReplicatedMergeTreeRestartingThread;
friend class ReplicatedMergeTreeCleanupThread;
@ -388,7 +391,6 @@ private:
*/
void waitForReplicaToProcessLogEntry(const String & replica_name, const LogEntry & entry);
/// Преобразовать число в строку формате суффиксов автоинкрементных нод в ZooKeeper.
static String padIndex(UInt64 index)
{

View File

@ -0,0 +1,38 @@
#pragma once
#include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h>
namespace DB
{
/** Реализует системную таблицу columns, которая позволяет получить информацию
* о столбцах каждой таблицы для всех баз данных.
*/
class StorageSystemColumns : public IStorage
{
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemColumns"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override;
private:
StorageSystemColumns(const std::string & name_);
private:
const std::string name;
NamesAndTypesList columns;
};
}

View File

@ -0,0 +1,38 @@
#pragma once
#include <DB/Storages/IStorage.h>
#include <DB/Interpreters/Context.h>
namespace DB
{
/** Реализует системную таблицу functions, которая позволяет получить список
* всех обычных и агрегатных функций.
*/
class StorageSystemFunctions : public IStorage
{
public:
static StoragePtr create(const std::string & name_);
std::string getName() const override { return "SystemFunctions"; }
std::string getTableName() const override { return name; }
const NamesAndTypesList & getColumnsListImpl() const override { return columns; }
BlockInputStreams read(
const Names & column_names,
ASTPtr query,
const Context & context,
const Settings & settings,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,
unsigned threads = 1) override;
private:
StorageSystemFunctions(const std::string & name_);
private:
const std::string name;
NamesAndTypesList columns;
};
}

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