From b18764404a680f706cd1081cb97d2c9a4cbce696 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 24 Jun 2015 14:03:53 +0300 Subject: [PATCH 01/50] Merge --- .../include/DB/Storages/MergeTree/MarkRange.h | 24 ++ .../MergeTree/MergeTreeDataSelectExecutor.h | 17 +- .../DB/Storages/MergeTree/MergeTreeReadPool.h | 257 ++++++++++++++ .../DB/Storages/MergeTree/MergeTreeReader.h | 14 +- .../MergeTreeThreadBlockInputStream.h | 328 ++++++++++++++++++ .../DB/Storages/MergeTree/RangesInDataPart.h | 29 ++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 97 +----- 7 files changed, 657 insertions(+), 109 deletions(-) create mode 100644 dbms/include/DB/Storages/MergeTree/MarkRange.h create mode 100644 dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h create mode 100644 dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h create mode 100644 dbms/include/DB/Storages/MergeTree/RangesInDataPart.h diff --git a/dbms/include/DB/Storages/MergeTree/MarkRange.h b/dbms/include/DB/Storages/MergeTree/MarkRange.h new file mode 100644 index 00000000000..c8258192a3c --- /dev/null +++ b/dbms/include/DB/Storages/MergeTree/MarkRange.h @@ -0,0 +1,24 @@ +#pragma once + +#include + + +namespace DB +{ + + +/** Пара засечек, определяющая диапазон строк в куске. Именно, диапазон имеет вид [begin * index_granularity, end * index_granularity). + */ +struct MarkRange +{ + std::size_t begin; + std::size_t end; + + MarkRange() = default; + MarkRange(const std::size_t begin, const std::size_t end) : begin{begin}, end{end} {} +}; + +using MarkRanges = std::vector; + + +} diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/dbms/include/DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h index f01519dece0..281de55aa66 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB @@ -33,22 +34,6 @@ private: Logger * log; - struct RangesInDataPart - { - MergeTreeData::DataPartPtr data_part; - size_t part_index_in_query; - MarkRanges ranges; - - RangesInDataPart() {} - - RangesInDataPart(MergeTreeData::DataPartPtr data_part_, size_t part_index_in_query_) - : data_part(data_part_), part_index_in_query(part_index_in_query_) - { - } - }; - - typedef std::vector RangesInDataParts; - BlockInputStreams spreadMarkRangesAmongThreads( RangesInDataParts parts, size_t threads, diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h new file mode 100644 index 00000000000..1759029ef3f --- /dev/null +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -0,0 +1,257 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + + +struct MergeTreeReadTask +{ + MergeTreeData::DataPartPtr data_part; + MarkRanges mark_ranges; + std::size_t part_index_in_query; + const Names & ordered_names; + const NameSet & column_name_set; + const NamesAndTypesList & columns; + const NamesAndTypesList & pre_columns; + const bool remove_prewhere_column; + + MergeTreeReadTask(const MergeTreeData::DataPartPtr & data_part, const MarkRanges & ranges, + const std::size_t part_index_in_query, const Names & ordered_names, + const NameSet & column_name_set, const NamesAndTypesList & columns, + const NamesAndTypesList & pre_columns, const bool remove_prewhere_column) + : data_part{data_part}, mark_ranges{ranges}, part_index_in_query{part_index_in_query}, + ordered_names{ordered_names}, column_name_set{column_name_set}, columns{columns}, pre_columns{pre_columns}, + remove_prewhere_column{remove_prewhere_column} + {} +}; + +using MergeTreeReadTaskPtr = std::unique_ptr; + +class MergeTreeReadPool +{ +public: + MergeTreeReadPool(const RangesInDataParts & parts, const std::vector & per_part_sum_marks, + const std::size_t sum_marks, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions, + const String & prewhere_column_name, const bool check_columns, const Names & column_names) + : parts{parts}, per_part_sum_marks{per_part_sum_marks}, sum_marks{sum_marks}, data{data} + { + fillPerPartInfo(column_names, prewhere_actions, prewhere_column_name, check_columns); + } + + MergeTreeReadPool(const MergeTreeReadPool &) = delete; + MergeTreeReadPool & operator=(const MergeTreeReadPool &) = delete; + + MergeTreeReadTaskPtr getTask(const std::size_t min_marks_to_read) + { + const std::lock_guard lock{mutex}; + + if (0 == sum_marks) + return nullptr; + + /// @todo use map to speedup lookup + /// find a part which has marks remaining + std::size_t part_id = 0; + for (; part_id < parts.size(); ++part_id) + if (0 != per_part_sum_marks[part_id]) + break; + + auto & part = parts[part_id]; + const auto & ordered_names = per_part_ordered_names[part_id]; + const auto & column_name_set = per_part_column_name_set[part_id]; + const auto & columns = per_part_columns[part_id]; + const auto & pre_columns = per_part_pre_columns[part_id]; + const auto remove_prewhere_column = per_part_remove_prewhere_column[part_id]; + auto & marks_in_part = per_part_sum_marks[part_id]; + + /// Берём весь кусок, если он достаточно мал + auto need_marks = std::min(marks_in_part, min_marks_to_read); + + /// Не будем оставлять в куске слишком мало строк. + if (marks_in_part > need_marks && + marks_in_part - need_marks < min_marks_to_read) + need_marks = marks_in_part; + + MarkRanges ranges_to_get_from_part; + + /// Возьмем весь кусок, если он достаточно мал. + if (marks_in_part <= need_marks) + { + const auto marks_to_get_from_range = marks_in_part; + + /// Восстановим порядок отрезков. + std::reverse(part.ranges.begin(), part.ranges.end()); + + ranges_to_get_from_part = part.ranges; + + marks_in_part -= marks_to_get_from_range; + sum_marks -= marks_to_get_from_range; + } + else + { + /// Цикл по отрезкам куска. + while (need_marks > 0 && !part.ranges.empty()) + { + auto & range = part.ranges.back(); + + const std::size_t marks_in_range = range.end - range.begin; + const std::size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); + + ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); + range.begin += marks_to_get_from_range; + if (range.begin == range.end) + part.ranges.pop_back(); + + marks_in_part -= marks_to_get_from_range; + need_marks -= marks_to_get_from_range; + sum_marks -= marks_to_get_from_range; + } + } + + return std::make_unique( + part.data_part, ranges_to_get_from_part, part.part_index_in_query, ordered_names, column_name_set, columns, + pre_columns, remove_prewhere_column); + } + +public: + void fillPerPartInfo(const Names & column_names, const ExpressionActionsPtr & prewhere_actions, + const String & prewhere_column_name, const bool check_columns) + { + for (const auto & part : parts) + { + per_part_columns_lock.push_back(std::make_unique( + part.data_part->columns_lock)); + + /// inject column names required for DEFAULT evaluation in current part + auto required_column_names = column_names; + + const auto injected_columns = injectRequiredColumns(part.data_part, required_column_names); + + /// insert injected columns into ordered columns list to avoid exception about different block structures + auto ordered_names = column_names; + ordered_names.insert(std::end(ordered_names), std::begin(injected_columns), std::end(injected_columns)); + per_part_ordered_names.emplace_back(ordered_names); + + Names required_pre_column_names; + + if (prewhere_actions) + { + /// collect columns required for PREWHERE evaluation + /// @todo minimum size column may be added here due to const condition, thus invalidating ordered_names + required_pre_column_names = prewhere_actions->getRequiredColumns(); + + /// there must be at least one column required for PREWHERE + if (required_pre_column_names.empty()) + required_pre_column_names.push_back(required_column_names[0]); + + /// PREWHERE columns may require some additional columns for DEFAULT evaluation + (void) injectRequiredColumns(part.data_part, required_pre_column_names); + + /// will be used to distinguish between PREWHERE and WHERE columns when applying filter + const NameSet pre_name_set{ + std::begin(required_pre_column_names), std::end(required_pre_column_names) + }; + /** Если выражение в PREWHERE - не столбец таблицы, не нужно отдавать наружу столбец с ним + * (от storage ожидают получить только столбцы таблицы). */ + per_part_remove_prewhere_column.push_back(0 == pre_name_set.count(prewhere_column_name)); + + Names post_column_names; + for (const auto & name : required_column_names) + if (!pre_name_set.count(name)) + post_column_names.push_back(name); + + required_column_names = post_column_names; + } + else + per_part_remove_prewhere_column.push_back(false); + + per_part_column_name_set.emplace_back(std::begin(required_column_names), std::end(required_column_names)); + + if (check_columns) + { + /** Под part->columns_lock проверим, что все запрошенные столбцы в куске того же типа, что в таблице. + * Это может быть не так во время ALTER MODIFY. */ + if (!required_pre_column_names.empty()) + data.check(part.data_part->columns, required_pre_column_names); + if (!required_column_names.empty()) + data.check(part.data_part->columns, required_column_names); + + per_part_pre_columns.push_back(data.getColumnsList().addTypes(required_pre_column_names)); + per_part_columns.push_back(data.getColumnsList().addTypes(required_column_names)); + } + else + { + per_part_pre_columns.push_back(part.data_part->columns.addTypes(required_pre_column_names)); + per_part_columns.push_back(part.data_part->columns.addTypes(required_column_names)); + } + } + } + + /** Если некоторых запрошенных столбцов нет в куске, + * то выясняем, какие столбцы может быть необходимо дополнительно прочитать, + * чтобы можно было вычислить DEFAULT выражение для этих столбцов. + * Добавляет их в columns. */ + NameSet injectRequiredColumns(const MergeTreeData::DataPartPtr & part, Names & columns) const + { + NameSet required_columns{std::begin(columns), std::end(columns)}; + NameSet injected_columns; + + for (size_t i = 0; i < columns.size(); ++i) + { + const auto & column_name = columns[i]; + + /// column has files and hence does not require evaluation + if (part->hasColumnFiles(column_name)) + continue; + + const auto default_it = data.column_defaults.find(column_name); + /// columns has no explicit default expression + if (default_it == std::end(data.column_defaults)) + continue; + + /// collect identifiers required for evaluation + IdentifierNameSet identifiers; + default_it->second.expression->collectIdentifierNames(identifiers); + + for (const auto & identifier : identifiers) + { + if (data.hasColumn(identifier)) + { + /// ensure each column is added only once + if (required_columns.count(identifier) == 0) + { + columns.emplace_back(identifier); + required_columns.emplace(identifier); + injected_columns.emplace(identifier); + } + } + } + } + + return injected_columns; + } + + std::vector> per_part_columns_lock; + RangesInDataParts parts; + std::vector per_part_sum_marks; + std::size_t sum_marks; + MergeTreeData & data; + std::vector per_part_ordered_names; + std::vector per_part_column_name_set; + std::vector per_part_columns; + std::vector per_part_pre_columns; + /// @todo actually all of these values are either true or false for the whole query, thus no vector required + std::vector per_part_remove_prewhere_column; + + mutable std::mutex mutex; +}; + +using MergeTreeReadPoolPtr = std::shared_ptr; + + +} diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index e5683f4caeb..c55e67412f1 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -17,19 +18,6 @@ namespace DB { -/** Пара засечек, определяющая диапазон строк в куске. Именно, диапазон имеет вид [begin * index_granularity, end * index_granularity). - */ -struct MarkRange -{ - size_t begin; - size_t end; - - MarkRange() {} - MarkRange(size_t begin_, size_t end_) : begin(begin_), end(end_) {} -}; - -typedef std::vector MarkRanges; - /** Умеет читать данные между парой засечек из одного куска. При чтении последовательных отрезков не делает лишних seek-ов. * При чтении почти последовательных отрезков делает seek-и быстро, не выбрасывая содержимое буфера. diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h new file mode 100644 index 00000000000..6cc887ebf0c --- /dev/null +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -0,0 +1,328 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + + +class MergeTreeThreadBlockInputStream : public IProfilingBlockInputStream +{ +public: + MergeTreeThreadBlockInputStream( + const MergeTreeReadPoolPtr & pool, const std::size_t min_marks_to_read, const std::size_t block_size, + MergeTreeData & storage, const bool use_uncompressed_cache, const ExpressionActionsPtr & prewhere_actions, + const String & prewhere_column, const std::size_t min_bytes_to_use_direct_io, + const std::size_t max_read_buffer_size, const Names & virt_column_names) + : pool{pool}, min_marks_to_read{min_marks_to_read}, block_size{block_size}, storage{storage}, + use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions}, + prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{min_bytes_to_use_direct_io}, + max_read_buffer_size{max_read_buffer_size}, virt_column_names{virt_column_names}, + log{&Logger::get("MergeTreeThreadBlockInputStream")} + {} + + String getName() const override { return "MergeTreeThread"; } + + String getID() const override + { + std::stringstream res; +// res << "MergeTreeThread(columns"; +// +// for (const auto & column : columns) +// res << ", " << column.name; +// +// if (prewhere_actions) +// res << ", prewhere, " << prewhere_actions->getID(); +// +// res << ", marks"; +// +// for (size_t i = 0; i < all_mark_ranges.size(); ++i) +// res << ", " << all_mark_ranges[i].begin << ", " << all_mark_ranges[i].end; +// +// res << ")"; + return res.str(); + } + +protected: + /// Будем вызывать progressImpl самостоятельно. + void progress(const Progress & value) override {} + + Block readImpl() override + { + Block res; + + while (true) + { + if (!task) + { + task = pool->getTask(min_marks_to_read); + + if (!task) + break; + + if (!initialized) + { + if (use_uncompressed_cache) + owned_uncompressed_cache = storage.context.getUncompressedCache(); + + owned_mark_cache = storage.context.getMarkCache(); + + initialized = true; + } + + const auto path = storage.getFullPath() + task->data_part->name + '/'; + + reader = std::make_unique( + path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), + storage, + task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); + + if (prewhere_actions) + pre_reader = std::make_unique( + path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), + owned_mark_cache.get(), + storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); + } + + res = readFromPart(); + + if (res) + { + const auto rows = res.rowsInFirstColumn(); + + /// add virtual columns + if (!virt_column_names.empty()) + { + + for (const auto & virt_column_name : virt_column_names) + { + if (virt_column_name == "_part") + { + res.insert(ColumnWithNameAndType{ + ColumnConst{rows, task->data_part->name}.convertToFullColumn(), + new DataTypeString, + virt_column_name + }); + } + else if (virt_column_name == "_part_index") + { + res.insert(ColumnWithNameAndType{ + ColumnConst{rows, task->part_index_in_query}.convertToFullColumn(), + new DataTypeUInt64, + virt_column_name + }); + } + } + } + } + + if (task->mark_ranges.empty()) + { + /** Закрываем файлы (ещё до уничтожения объекта). + * Чтобы при создании многих источников, но одновременном чтении только из нескольких, + * буферы не висели в памяти. */ + task = {}; + reader = {}; + pre_reader = {}; + } + + if (res) + break; + } + + return res; + } + +private: + Block readFromPart() + { + Block res; + + if (prewhere_actions) + { + do + { + /// Прочитаем полный блок столбцов, нужных для вычисления выражения в PREWHERE. + size_t space_left = std::max(1LU, block_size / storage.index_granularity); + MarkRanges ranges_to_read; + while (!task->mark_ranges.empty() && space_left) + { + auto & range = task->mark_ranges.back(); + + size_t marks_to_read = std::min(range.end - range.begin, space_left); + pre_reader->readRange(range.begin, range.begin + marks_to_read, res); + + ranges_to_read.emplace_back(range.begin, range.begin + marks_to_read); + space_left -= marks_to_read; + range.begin += marks_to_read; + if (range.begin == range.end) + task->mark_ranges.pop_back(); + } + progressImpl({ res.rowsInFirstColumn(), res.bytes() }); + pre_reader->fillMissingColumns(res, task->ordered_names); + + /// Вычислим выражение в PREWHERE. + prewhere_actions->execute(res); + + ColumnPtr column = res.getByName(prewhere_column).column; + if (task->remove_prewhere_column) + res.erase(prewhere_column); + + const auto pre_bytes = res.bytes(); + + /** Если фильтр - константа (например, написано PREWHERE 1), + * то либо вернём пустой блок, либо вернём блок без изменений. + */ + if (const auto column_const = typeid_cast(column.get())) + { + if (!column_const->getData()) + { + res.clear(); + return res; + } + + for (const auto & range : ranges_to_read) + reader->readRange(range.begin, range.end, res); + + progressImpl({ 0, res.bytes() - pre_bytes }); + } + else if (const auto column_vec = typeid_cast(column.get())) + { + size_t index_granularity = storage.index_granularity; + + const auto & pre_filter = column_vec->getData(); + IColumn::Filter post_filter(pre_filter.size()); + + /// Прочитаем в нужных отрезках остальные столбцы и составим для них свой фильтр. + size_t pre_filter_pos = 0; + size_t post_filter_pos = 0; + + for (const auto & range : ranges_to_read) + { + auto begin = range.begin; + auto pre_filter_begin_pos = pre_filter_pos; + + for (auto mark = range.begin; mark <= range.end; ++mark) + { + UInt8 nonzero = 0; + + if (mark != range.end) + { + const size_t limit = std::min(pre_filter.size(), pre_filter_pos + index_granularity); + for (size_t row = pre_filter_pos; row < limit; ++row) + nonzero |= pre_filter[row]; + } + + if (!nonzero) + { + if (mark > begin) + { + memcpy( + &post_filter[post_filter_pos], + &pre_filter[pre_filter_begin_pos], + pre_filter_pos - pre_filter_begin_pos); + post_filter_pos += pre_filter_pos - pre_filter_begin_pos; + reader->readRange(begin, mark, res); + } + begin = mark + 1; + pre_filter_begin_pos = std::min(pre_filter_pos + index_granularity, pre_filter.size()); + } + + if (mark < range.end) + pre_filter_pos = std::min(pre_filter_pos + index_granularity, pre_filter.size()); + } + } + + if (!post_filter_pos) + { + res.clear(); + continue; + } + + progressImpl({ 0, res.bytes() - pre_bytes }); + + post_filter.resize(post_filter_pos); + + /// Отфильтруем столбцы, относящиеся к PREWHERE, используя pre_filter, + /// остальные столбцы - используя post_filter. + size_t rows = 0; + for (const auto i : ext::range(0, res.columns())) + { + auto & col = res.getByPosition(i); + if (col.name == prewhere_column && res.columns() > 1) + continue; + col.column = + col.column->filter(task->column_name_set.count(col.name) ? post_filter : pre_filter); + rows = col.column->size(); + } + + /// Заменим столбец со значением условия из PREWHERE на константу. + if (!task->remove_prewhere_column) + res.getByName(prewhere_column).column = new ColumnConstUInt8{rows, 1}; + } + else + throw Exception{ + "Illegal type " + column->getName() + " of column for filter. Must be ColumnUInt8 or ColumnConstUInt8.", + ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER + }; + + reader->fillMissingColumnsAndReorder(res, task->ordered_names); + } + while (!task->mark_ranges.empty() && !res && !isCancelled()); + } + else + { + size_t space_left = std::max(1LU, block_size / storage.index_granularity); + + while (!task->mark_ranges.empty() && space_left) + { + auto & range = task->mark_ranges.back(); + + const size_t marks_to_read = std::min(range.end - range.begin, space_left); + reader->readRange(range.begin, range.begin + marks_to_read, res); + + space_left -= marks_to_read; + range.begin += marks_to_read; + if (range.begin == range.end) + task->mark_ranges.pop_back(); + } + + progressImpl({ res.rowsInFirstColumn(), res.bytes() }); + + reader->fillMissingColumns(res, task->ordered_names); + } + + return res; + } + + MergeTreeReadPoolPtr pool; + const std::size_t min_marks_to_read; + const std::size_t block_size; + MergeTreeData & storage; + const bool use_uncompressed_cache; + ExpressionActionsPtr prewhere_actions; + const String prewhere_column; + const std::size_t min_bytes_to_use_direct_io; + const std::size_t max_read_buffer_size; + const Names virt_column_names; + + Logger * log; + + using MergeTreeReaderPtr = std::unique_ptr; + + bool initialized; + UncompressedCachePtr owned_uncompressed_cache; + MarkCachePtr owned_mark_cache; + + MergeTreeReadTaskPtr task; + MergeTreeReaderPtr reader; + MergeTreeReaderPtr pre_reader; +}; + + +} diff --git a/dbms/include/DB/Storages/MergeTree/RangesInDataPart.h b/dbms/include/DB/Storages/MergeTree/RangesInDataPart.h new file mode 100644 index 00000000000..7bfefa47fda --- /dev/null +++ b/dbms/include/DB/Storages/MergeTree/RangesInDataPart.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + + +namespace DB +{ + + +struct RangesInDataPart +{ + MergeTreeData::DataPartPtr data_part; + std::size_t part_index_in_query; + MarkRanges ranges; + + RangesInDataPart() = default; + + RangesInDataPart(const MergeTreeData::DataPartPtr & data_part, const std::size_t part_index_in_query, + const MarkRanges & ranges = MarkRanges{}) + : data_part{data_part}, part_index_in_query{part_index_in_query}, ranges{ranges} + { + } +}; + +using RangesInDataParts = std::vector; + + +} diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 5bb155a3b06..7a112494ff4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1,6 +1,7 @@ #include #include -#include +#include +#include #include #include #include @@ -8,7 +9,6 @@ #include #include #include -#include #include @@ -340,9 +340,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( const Names & virt_columns, const Settings & settings) { - const size_t min_marks_for_concurrent_read = + const std::size_t min_marks_for_concurrent_read = (settings.merge_tree_min_rows_for_concurrent_read + data.index_granularity - 1) / data.index_granularity; - const size_t max_marks_to_use_cache = + const std::size_t max_marks_to_use_cache = (settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity; /// На всякий случай перемешаем куски. @@ -365,89 +365,26 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; + MergeTreeReadPoolPtr pool = std::make_shared( + parts, sum_marks_in_parts, sum_marks, data, prewhere_actions, prewhere_column, true, column_names); + BlockInputStreams res; if (sum_marks > 0) { - const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; + for (std::size_t i = 0; i < threads; ++i) + res.emplace_back(new MergeTreeThreadBlockInputStream{ + pool, min_marks_for_concurrent_read, max_block_size, data, use_uncompressed_cache, prewhere_actions, + prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns + }); - for (size_t i = 0; i < threads && !parts.empty(); ++i) - { - size_t need_marks = min_marks_per_thread; + /// Оценим общее количество строк - для прогресс-бара. + const std::size_t total_rows = data.index_granularity * sum_marks; - /// Цикл по кускам. - while (need_marks > 0 && !parts.empty()) - { - RangesInDataPart & part = parts.back(); - size_t & marks_in_part = sum_marks_in_parts.back(); + /// Выставим приблизительное количество строк только для первого источника + static_cast(*res.front()).setTotalRowsApprox(total_rows); - /// Не будем брать из куска слишком мало строк. - if (marks_in_part >= min_marks_for_concurrent_read && - need_marks < min_marks_for_concurrent_read) - need_marks = min_marks_for_concurrent_read; - - /// Не будем оставлять в куске слишком мало строк. - if (marks_in_part > need_marks && - marks_in_part - need_marks < min_marks_for_concurrent_read) - need_marks = marks_in_part; - - MarkRanges ranges_to_get_from_part; - - /// Возьмем весь кусок, если он достаточно мал. - if (marks_in_part <= need_marks) - { - /// Восстановим порядок отрезков. - std::reverse(part.ranges.begin(), part.ranges.end()); - - ranges_to_get_from_part = part.ranges; - - need_marks -= marks_in_part; - parts.pop_back(); - sum_marks_in_parts.pop_back(); - } - else - { - /// Цикл по отрезкам куска. - while (need_marks > 0) - { - if (part.ranges.empty()) - throw Exception("Unexpected end of ranges while spreading marks among threads", ErrorCodes::LOGICAL_ERROR); - - MarkRange & range = part.ranges.back(); - - const size_t marks_in_range = range.end - range.begin; - const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); - - ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); - range.begin += marks_to_get_from_range; - marks_in_part -= marks_to_get_from_range; - need_marks -= marks_to_get_from_range; - if (range.begin == range.end) - part.ranges.pop_back(); - } - } - - BlockInputStreamPtr source_stream = new MergeTreeBlockInputStream( - data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data, - part.data_part, ranges_to_get_from_part, use_uncompressed_cache, - prewhere_actions, prewhere_column, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size); - - res.push_back(source_stream); - - for (const String & virt_column : virt_columns) - { - if (virt_column == "_part") - res.back() = new AddingConstColumnBlockInputStream( - res.back(), new DataTypeString, part.data_part->name, "_part"); - else if (virt_column == "_part_index") - res.back() = new AddingConstColumnBlockInputStream( - res.back(), new DataTypeUInt64, part.part_index_in_query, "_part_index"); - } - } - } - - if (!parts.empty()) - throw Exception("Couldn't spread marks among threads", ErrorCodes::LOGICAL_ERROR); + LOG_TRACE(log, "Reading approx. " << total_rows); } return res; From aa7fe6396214e0794a25b0974ec07f695501c7ef Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 24 Jun 2015 15:21:43 +0300 Subject: [PATCH 02/50] dbms: temporarily add merge_tree_uniform_read_distribution setting. [#METR-16457] --- dbms/include/DB/Interpreters/Settings.h | 2 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 85 ++++++++++++++++++- 2 files changed, 86 insertions(+), 1 deletion(-) diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 8d799249f71..0b7983efde3 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -109,6 +109,8 @@ struct Settings * (Чтобы большие запросы не вымывали кэш.) */ \ M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \ \ + M(SettingBool, merge_tree_uniform_read_distribution, false) \ + \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \ \ diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 7a112494ff4..2a659373af9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -370,7 +370,90 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( BlockInputStreams res; - if (sum_marks > 0) + /// @todo remove old code + if (sum_marks > 0 && settings.merge_tree_uniform_read_distribution == 0) + { + const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; + + for (size_t i = 0; i < threads && !parts.empty(); ++i) + { + size_t need_marks = min_marks_per_thread; + + /// Цикл по кускам. + while (need_marks > 0 && !parts.empty()) + { + RangesInDataPart & part = parts.back(); + size_t & marks_in_part = sum_marks_in_parts.back(); + + /// Не будем брать из куска слишком мало строк. + if (marks_in_part >= min_marks_for_concurrent_read && + need_marks < min_marks_for_concurrent_read) + need_marks = min_marks_for_concurrent_read; + + /// Не будем оставлять в куске слишком мало строк. + if (marks_in_part > need_marks && + marks_in_part - need_marks < min_marks_for_concurrent_read) + need_marks = marks_in_part; + + MarkRanges ranges_to_get_from_part; + + /// Возьмем весь кусок, если он достаточно мал. + if (marks_in_part <= need_marks) + { + /// Восстановим порядок отрезков. + std::reverse(part.ranges.begin(), part.ranges.end()); + + ranges_to_get_from_part = part.ranges; + + need_marks -= marks_in_part; + parts.pop_back(); + sum_marks_in_parts.pop_back(); + } + else + { + /// Цикл по отрезкам куска. + while (need_marks > 0) + { + if (part.ranges.empty()) + throw Exception("Unexpected end of ranges while spreading marks among threads", ErrorCodes::LOGICAL_ERROR); + + MarkRange & range = part.ranges.back(); + + const size_t marks_in_range = range.end - range.begin; + const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); + + ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); + range.begin += marks_to_get_from_range; + marks_in_part -= marks_to_get_from_range; + need_marks -= marks_to_get_from_range; + if (range.begin == range.end) + part.ranges.pop_back(); + } + } + + BlockInputStreamPtr source_stream = new MergeTreeBlockInputStream( + data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data, + part.data_part, ranges_to_get_from_part, use_uncompressed_cache, + prewhere_actions, prewhere_column, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size); + + res.push_back(source_stream); + + for (const String & virt_column : virt_columns) + { + if (virt_column == "_part") + res.back() = new AddingConstColumnBlockInputStream( + res.back(), new DataTypeString, part.data_part->name, "_part"); + else if (virt_column == "_part_index") + res.back() = new AddingConstColumnBlockInputStream( + res.back(), new DataTypeUInt64, part.part_index_in_query, "_part_index"); + } + } + } + + if (!parts.empty()) + throw Exception("Couldn't spread marks among threads", ErrorCodes::LOGICAL_ERROR); + } + else if (sum_marks > 0) { for (std::size_t i = 0; i < threads; ++i) res.emplace_back(new MergeTreeThreadBlockInputStream{ From 54d55177579da16bdf3bf44768dda56db53a2994 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 24 Jun 2015 20:05:40 +0300 Subject: [PATCH 03/50] Merge --- .../DB/Storages/MergeTree/MergeTreeReadPool.h | 11 +++++++--- .../DB/Storages/MergeTree/MergeTreeReader.h | 18 +++++++++-------- .../MergeTreeThreadBlockInputStream.h | 20 ++++++++++--------- 3 files changed, 29 insertions(+), 20 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index 1759029ef3f..d4ea74c7a4d 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -20,14 +20,16 @@ struct MergeTreeReadTask const NamesAndTypesList & columns; const NamesAndTypesList & pre_columns; const bool remove_prewhere_column; + const MarkRanges & all_ranges; MergeTreeReadTask(const MergeTreeData::DataPartPtr & data_part, const MarkRanges & ranges, const std::size_t part_index_in_query, const Names & ordered_names, const NameSet & column_name_set, const NamesAndTypesList & columns, - const NamesAndTypesList & pre_columns, const bool remove_prewhere_column) + const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, + const MarkRanges & all_ranges) : data_part{data_part}, mark_ranges{ranges}, part_index_in_query{part_index_in_query}, ordered_names{ordered_names}, column_name_set{column_name_set}, columns{columns}, pre_columns{pre_columns}, - remove_prewhere_column{remove_prewhere_column} + remove_prewhere_column{remove_prewhere_column}, all_ranges{all_ranges} {} }; @@ -115,7 +117,7 @@ public: return std::make_unique( part.data_part, ranges_to_get_from_part, part.part_index_in_query, ordered_names, column_name_set, columns, - pre_columns, remove_prewhere_column); + pre_columns, remove_prewhere_column, per_part_all_ranges[part_id]); } public: @@ -124,6 +126,8 @@ public: { for (const auto & part : parts) { + per_part_all_ranges.push_back(part.ranges); + per_part_columns_lock.push_back(std::make_unique( part.data_part->columns_lock)); @@ -238,6 +242,7 @@ public: std::vector> per_part_columns_lock; RangesInDataParts parts; + std::vector per_part_all_ranges; std::vector per_part_sum_marks; std::size_t sum_marks; MergeTreeData & data; diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index c55e67412f1..64aeb1076d5 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -203,13 +203,15 @@ public: fillMissingColumnsImpl(res, ordered_names, true); } + const MergeTreeData::DataPartPtr & getDataPart() const { return data_part; } + private: struct Stream { MarkCache::MappedPtr marks; ReadBuffer * data_buffer; - Poco::SharedPtr cached_buffer; - Poco::SharedPtr non_cached_buffer; + std::unique_ptr cached_buffer; + std::unique_ptr non_cached_buffer; std::string path_prefix; size_t max_mark_range; @@ -269,15 +271,15 @@ private: if (uncompressed_cache) { - cached_buffer = new CachedCompressedReadBuffer(path_prefix + ".bin", uncompressed_cache, - estimated_size, aio_threshold, buffer_size); - data_buffer = &*cached_buffer; + cached_buffer = std::make_unique( + path_prefix + ".bin", uncompressed_cache, estimated_size, aio_threshold, buffer_size); + data_buffer = cached_buffer.get(); } else { - non_cached_buffer = new CompressedReadBufferFromFile(path_prefix + ".bin", estimated_size, - aio_threshold, buffer_size); - data_buffer = &*non_cached_buffer; + non_cached_buffer = std::make_unique( + path_prefix + ".bin", estimated_size, aio_threshold, buffer_size); + data_buffer = non_cached_buffer.get(); } } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 6cc887ebf0c..a5330cfaf46 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -77,16 +77,18 @@ protected: const auto path = storage.getFullPath() + task->data_part->name + '/'; - reader = std::make_unique( - path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), - storage, - task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); + if (!reader || reader->getDataPart() != task->data_part) + { + reader = std::make_unique( + path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), + storage, task->all_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); - if (prewhere_actions) - pre_reader = std::make_unique( - path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), - owned_mark_cache.get(), - storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); + if (prewhere_actions) + pre_reader = std::make_unique( + path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), + owned_mark_cache.get(), storage, task->all_ranges, min_bytes_to_use_direct_io, + max_read_buffer_size); + } } res = readFromPart(); From a5bfc7092088de189676b8f869462b0144db961d Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Fri, 3 Jul 2015 18:08:21 +0300 Subject: [PATCH 04/50] Merge --- .../DB/Storages/MergeTree/MergeTreeReadPool.h | 139 +++++++++++++----- .../DB/Storages/MergeTree/MergeTreeReader.h | 95 +----------- .../MergeTreeThreadBlockInputStream.h | 135 ++++++++--------- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 4 files changed, 178 insertions(+), 193 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index d4ea74c7a4d..a522aca3879 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -20,16 +20,15 @@ struct MergeTreeReadTask const NamesAndTypesList & columns; const NamesAndTypesList & pre_columns; const bool remove_prewhere_column; - const MarkRanges & all_ranges; + const bool should_reorder; - MergeTreeReadTask(const MergeTreeData::DataPartPtr & data_part, const MarkRanges & ranges, - const std::size_t part_index_in_query, const Names & ordered_names, - const NameSet & column_name_set, const NamesAndTypesList & columns, - const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, - const MarkRanges & all_ranges) + MergeTreeReadTask( + const MergeTreeData::DataPartPtr & data_part, const MarkRanges & ranges, const std::size_t part_index_in_query, + const Names & ordered_names, const NameSet & column_name_set, const NamesAndTypesList & columns, + const NamesAndTypesList & pre_columns, const bool remove_prewhere_column, const bool should_reorder) : data_part{data_part}, mark_ranges{ranges}, part_index_in_query{part_index_in_query}, ordered_names{ordered_names}, column_name_set{column_name_set}, columns{columns}, pre_columns{pre_columns}, - remove_prewhere_column{remove_prewhere_column}, all_ranges{all_ranges} + remove_prewhere_column{remove_prewhere_column}, should_reorder{should_reorder} {} }; @@ -38,12 +37,12 @@ using MergeTreeReadTaskPtr = std::unique_ptr; class MergeTreeReadPool { public: - MergeTreeReadPool(const RangesInDataParts & parts, const std::vector & per_part_sum_marks, - const std::size_t sum_marks, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions, - const String & prewhere_column_name, const bool check_columns, const Names & column_names) - : parts{parts}, per_part_sum_marks{per_part_sum_marks}, sum_marks{sum_marks}, data{data} + MergeTreeReadPool( + const RangesInDataParts & parts, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions, + const String & prewhere_column_name, const bool check_columns, const Names & column_names) + : parts{parts}, data{data}, column_names{column_names} { - fillPerPartInfo(column_names, prewhere_actions, prewhere_column_name, check_columns); + fillPerPartInfo(prewhere_actions, prewhere_column_name, check_columns); } MergeTreeReadPool(const MergeTreeReadPool &) = delete; @@ -53,18 +52,13 @@ public: { const std::lock_guard lock{mutex}; - if (0 == sum_marks) + if (remaining_part_indices.empty()) return nullptr; - /// @todo use map to speedup lookup /// find a part which has marks remaining - std::size_t part_id = 0; - for (; part_id < parts.size(); ++part_id) - if (0 != per_part_sum_marks[part_id]) - break; + const auto part_id = remaining_part_indices.back(); auto & part = parts[part_id]; - const auto & ordered_names = per_part_ordered_names[part_id]; const auto & column_name_set = per_part_column_name_set[part_id]; const auto & columns = per_part_columns[part_id]; const auto & pre_columns = per_part_pre_columns[part_id]; @@ -92,7 +86,8 @@ public: ranges_to_get_from_part = part.ranges; marks_in_part -= marks_to_get_from_range; - sum_marks -= marks_to_get_from_range; + + remaining_part_indices.pop_back(); } else { @@ -111,22 +106,39 @@ public: marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; - sum_marks -= marks_to_get_from_range; } + + if (0 == marks_in_part) + remaining_part_indices.pop_back(); } return std::make_unique( - part.data_part, ranges_to_get_from_part, part.part_index_in_query, ordered_names, column_name_set, columns, - pre_columns, remove_prewhere_column, per_part_all_ranges[part_id]); + part.data_part, ranges_to_get_from_part, part.part_index_in_query, column_names, column_name_set, columns, + pre_columns, remove_prewhere_column, per_part_should_reorder[part_id]); } public: - void fillPerPartInfo(const Names & column_names, const ExpressionActionsPtr & prewhere_actions, - const String & prewhere_column_name, const bool check_columns) + void fillPerPartInfo( + const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column_name, const bool check_columns) { - for (const auto & part : parts) + remaining_part_indices.reserve(parts.size()); + + for (const auto i : ext::range(0, parts.size())) { - per_part_all_ranges.push_back(part.ranges); + auto & part = parts[i]; + + /// Посчитаем засечки для каждого куска. + size_t sum_marks = 0; + /// Пусть отрезки будут перечислены справа налево, чтобы можно было выбрасывать самый левый отрезок с помощью pop_back(). + std::reverse(std::begin(part.ranges), std::end(part.ranges)); + + for (const auto & range : part.ranges) + sum_marks += range.end - range.begin; + + per_part_sum_marks.push_back(sum_marks); + + if (0 != sum_marks) + remaining_part_indices.push_back(i); per_part_columns_lock.push_back(std::make_unique( part.data_part->columns_lock)); @@ -135,18 +147,13 @@ public: auto required_column_names = column_names; const auto injected_columns = injectRequiredColumns(part.data_part, required_column_names); - - /// insert injected columns into ordered columns list to avoid exception about different block structures - auto ordered_names = column_names; - ordered_names.insert(std::end(ordered_names), std::begin(injected_columns), std::end(injected_columns)); - per_part_ordered_names.emplace_back(ordered_names); + auto should_reoder = !injected_columns.empty(); Names required_pre_column_names; if (prewhere_actions) { /// collect columns required for PREWHERE evaluation - /// @todo minimum size column may be added here due to const condition, thus invalidating ordered_names required_pre_column_names = prewhere_actions->getRequiredColumns(); /// there must be at least one column required for PREWHERE @@ -154,7 +161,9 @@ public: required_pre_column_names.push_back(required_column_names[0]); /// PREWHERE columns may require some additional columns for DEFAULT evaluation - (void) injectRequiredColumns(part.data_part, required_pre_column_names); + const auto injected_pre_columns = injectRequiredColumns(part.data_part, required_pre_column_names); + if (!injected_pre_columns.empty()) + should_reoder = true; /// will be used to distinguish between PREWHERE and WHERE columns when applying filter const NameSet pre_name_set{ @@ -193,6 +202,8 @@ public: per_part_pre_columns.push_back(part.data_part->columns.addTypes(required_pre_column_names)); per_part_columns.push_back(part.data_part->columns.addTypes(required_column_names)); } + + per_part_should_reorder.push_back(should_reoder); } } @@ -205,13 +216,18 @@ public: NameSet required_columns{std::begin(columns), std::end(columns)}; NameSet injected_columns; + auto all_column_files_missing = true; + for (size_t i = 0; i < columns.size(); ++i) { const auto & column_name = columns[i]; /// column has files and hence does not require evaluation if (part->hasColumnFiles(column_name)) + { + all_column_files_missing = false; continue; + } const auto default_it = data.column_defaults.find(column_name); /// columns has no explicit default expression @@ -237,21 +253,70 @@ public: } } + if (all_column_files_missing) + { + addMinimumSizeColumn(part, columns); + /// correctly report added column + injected_columns.insert(columns.back()); + } + return injected_columns; } + /** Добавить столбец минимального размера. + * Используется в случае, когда ни один столбец не нужен, но нужно хотя бы знать количество строк. + * Добавляет в columns. + */ + void addMinimumSizeColumn(const MergeTreeData::DataPartPtr & part, Names & columns) const + { + const auto get_column_size = [this, &part] (const String & name) { + const auto & files = part->checksums.files; + + const auto escaped_name = escapeForFileName(name); + const auto bin_file_name = escaped_name + ".bin"; + const auto mrk_file_name = escaped_name + ".mrk"; + + return files.find(bin_file_name)->second.file_size + files.find(mrk_file_name)->second.file_size; + }; + + const auto & storage_columns = data.getColumnsList(); + const NameAndTypePair * minimum_size_column = nullptr; + auto minimum_size = std::numeric_limits::max(); + + for (const auto & column : storage_columns) + { + if (!part->hasColumnFiles(column.name)) + continue; + + const auto size = get_column_size(column.name); + if (size < minimum_size) + { + minimum_size = size; + minimum_size_column = &column; + } + } + + if (!minimum_size_column) + throw Exception{ + "Could not find a column of minimum size in MergeTree", + ErrorCodes::LOGICAL_ERROR + }; + + columns.push_back(minimum_size_column->name); + } + std::vector> per_part_columns_lock; RangesInDataParts parts; - std::vector per_part_all_ranges; std::vector per_part_sum_marks; - std::size_t sum_marks; + std::vector remaining_part_indices; MergeTreeData & data; - std::vector per_part_ordered_names; + Names column_names; std::vector per_part_column_name_set; std::vector per_part_columns; std::vector per_part_pre_columns; /// @todo actually all of these values are either true or false for the whole query, thus no vector required std::vector per_part_remove_prewhere_column; + std::vector per_part_should_reorder; mutable std::mutex mutex; }; diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index 64aeb1076d5..9c799c1828b 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -62,20 +62,14 @@ public: { size_t max_rows_to_read = (to_mark - from_mark) * storage.index_granularity; - /** Для некоторых столбцов файлы с данными могут отсутствовать. - * Это бывает для старых кусков, после добавления новых столбцов в структуру таблицы. - */ - auto has_missing_columns = false; - /// Указатели на столбцы смещений, общие для столбцов из вложенных структур данных /// Если append, все значения nullptr, и offset_columns используется только для проверки, что столбец смещений уже прочитан. OffsetColumns offset_columns; - const auto read_column = [&] (const NameAndTypePair & it) { + + for (const NameAndTypePair & it : columns) + { if (streams.end() == streams.find(it.name)) - { - has_missing_columns = true; - return; - } + continue; /// Все столбцы уже есть в блоке. Будем добавлять значения в конец. bool append = res.has(it.name); @@ -108,24 +102,12 @@ public: if (!append && column.column->size()) res.insert(column); - }; - - for (const NameAndTypePair & it : columns) - read_column(it); - - if (has_missing_columns && !res) - { - addMinimumSizeColumn(); - /// minimum size column is necessarily at list's front - read_column(columns.front()); } } catch (const Exception & e) { if (e.code() != ErrorCodes::MEMORY_LIMIT_EXCEEDED) - { storage.reportBrokenPart(part_name); - } /// Более хорошая диагностика. throw Exception(e.message() + "\n(while reading from part " + path + " from mark " + toString(from_mark) + " to " @@ -139,60 +121,13 @@ public: } } - - /** Добавить столбец минимального размера. - * Используется в случае, когда ни один столбец не нужен, но нужно хотя бы знать количество строк. - * Добавляет в columns. - */ - void addMinimumSizeColumn() - { - const auto get_column_size = [this] (const String & name) { - const auto & files = data_part->checksums.files; - - const auto escaped_name = escapeForFileName(name); - const auto bin_file_name = escaped_name + ".bin"; - const auto mrk_file_name = escaped_name + ".mrk"; - - return files.find(bin_file_name)->second.file_size + files.find(mrk_file_name)->second.file_size; - }; - - const auto & storage_columns = storage.getColumnsList(); - const NameAndTypePair * minimum_size_column = nullptr; - auto minimum_size = std::numeric_limits::max(); - - for (const auto & column : storage_columns) - { - if (!data_part->hasColumnFiles(column.name)) - continue; - - const auto size = get_column_size(column.name); - if (size < minimum_size) - { - minimum_size = size; - minimum_size_column = &column; - } - } - - if (!minimum_size_column) - throw Exception{ - "could not find a column of minimum size in MergeTree", - ErrorCodes::LOGICAL_ERROR - }; - - addStream(minimum_size_column->name, *minimum_size_column->type, all_mark_ranges); - columns.emplace(std::begin(columns), *minimum_size_column); - - added_minimum_size_column = &columns.front(); - } - - /** Добавляет в блок недостающие столбцы из ordered_names, состоящие из значений по-умолчанию. * Недостающие столбцы добавляются в позиции, такие же как в ordered_names. * Если был добавлен хотя бы один столбец - то все столбцы в блоке переупорядочиваются как в ordered_names. */ - void fillMissingColumns(Block & res, const Names & ordered_names) + void fillMissingColumns(Block & res, const Names & ordered_names, const bool always_reorder = false) { - fillMissingColumnsImpl(res, ordered_names, false); + fillMissingColumnsImpl(res, ordered_names, always_reorder); } /** То же самое, но всегда переупорядочивает столбцы в блоке, как в ordered_names @@ -345,9 +280,8 @@ private: String part_name; FileStreams streams; - /// Запрашиваемые столбцы. Возможно, с добавлением minimum_size_column. + /// Запрашиваемые столбцы. NamesAndTypesList columns; - const NameAndTypePair * added_minimum_size_column = nullptr; UncompressedCache * uncompressed_cache; MarkCache * mark_cache; @@ -538,15 +472,6 @@ private: if (should_evaluate_defaults) evaluateMissingDefaults(res, columns, storage.column_defaults, storage.context); - /// remove added column to ensure same content among all blocks - if (added_minimum_size_column) - { - res.erase(0); - streams.erase(added_minimum_size_column->name); - columns.erase(std::begin(columns)); - added_minimum_size_column = nullptr; - } - /// sort columns to ensure consistent order among all blocks if (should_sort) { @@ -556,12 +481,6 @@ private: if (res.has(name)) ordered_block.insert(res.getByName(name)); - if (res.columns() != ordered_block.columns()) - throw Exception{ - "Ordered block has different number of columns than original one:\n" + - ordered_block.dumpNames() + "\nvs.\n" + res.dumpNames(), - ErrorCodes::LOGICAL_ERROR}; - std::swap(res, ordered_block); } } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index a5330cfaf46..9c403bc673d 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -56,72 +56,15 @@ protected: { Block res; - while (true) + while (!res) { - if (!task) - { - task = pool->getTask(min_marks_to_read); - - if (!task) - break; - - if (!initialized) - { - if (use_uncompressed_cache) - owned_uncompressed_cache = storage.context.getUncompressedCache(); - - owned_mark_cache = storage.context.getMarkCache(); - - initialized = true; - } - - const auto path = storage.getFullPath() + task->data_part->name + '/'; - - if (!reader || reader->getDataPart() != task->data_part) - { - reader = std::make_unique( - path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), - storage, task->all_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); - - if (prewhere_actions) - pre_reader = std::make_unique( - path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), - owned_mark_cache.get(), storage, task->all_ranges, min_bytes_to_use_direct_io, - max_read_buffer_size); - } - } + if (!task && !getNewTask()) + break; res = readFromPart(); if (res) - { - const auto rows = res.rowsInFirstColumn(); - - /// add virtual columns - if (!virt_column_names.empty()) - { - - for (const auto & virt_column_name : virt_column_names) - { - if (virt_column_name == "_part") - { - res.insert(ColumnWithNameAndType{ - ColumnConst{rows, task->data_part->name}.convertToFullColumn(), - new DataTypeString, - virt_column_name - }); - } - else if (virt_column_name == "_part_index") - { - res.insert(ColumnWithNameAndType{ - ColumnConst{rows, task->part_index_in_query}.convertToFullColumn(), - new DataTypeUInt64, - virt_column_name - }); - } - } - } - } + injectVirtualColumns(res); if (task->mark_ranges.empty()) { @@ -132,15 +75,44 @@ protected: reader = {}; pre_reader = {}; } - - if (res) - break; } return res; } private: + bool getNewTask() + { + task = pool->getTask(min_marks_to_read); + + if (!task) + return false; + + if (!initialized) + { + if (use_uncompressed_cache) + owned_uncompressed_cache = storage.context.getUncompressedCache(); + + owned_mark_cache = storage.context.getMarkCache(); + + initialized = true; + } + + const auto path = storage.getFullPath() + task->data_part->name + '/'; + + reader = std::make_unique( + path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), + storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); + + if (prewhere_actions) + pre_reader = std::make_unique( + path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), + owned_mark_cache.get(), storage, task->mark_ranges, min_bytes_to_use_direct_io, + max_read_buffer_size); + + return true; + } + Block readFromPart() { Block res; @@ -166,7 +138,7 @@ private: task->mark_ranges.pop_back(); } progressImpl({ res.rowsInFirstColumn(), res.bytes() }); - pre_reader->fillMissingColumns(res, task->ordered_names); + pre_reader->fillMissingColumns(res, task->ordered_names, task->should_reorder); /// Вычислим выражение в PREWHERE. prewhere_actions->execute(res); @@ -296,12 +268,41 @@ private: progressImpl({ res.rowsInFirstColumn(), res.bytes() }); - reader->fillMissingColumns(res, task->ordered_names); + reader->fillMissingColumns(res, task->ordered_names, task->should_reorder); } return res; } + void injectVirtualColumns(Block & block) + { + const auto rows = block.rowsInFirstColumn(); + + /// add virtual columns + if (!virt_column_names.empty()) + { + for (const auto & virt_column_name : virt_column_names) + { + if (virt_column_name == "_part") + { + block.insert(ColumnWithNameAndType{ + ColumnConst{rows, task->data_part->name}.convertToFullColumn(), + new DataTypeString, + virt_column_name + }); + } + else if (virt_column_name == "_part_index") + { + block.insert(ColumnWithNameAndType{ + ColumnConst{rows, task->part_index_in_query}.convertToFullColumn(), + new DataTypeUInt64, + virt_column_name + }); + } + } + } + } + MergeTreeReadPoolPtr pool; const std::size_t min_marks_to_read; const std::size_t block_size; @@ -317,7 +318,7 @@ private: using MergeTreeReaderPtr = std::unique_ptr; - bool initialized; + bool initialized{false}; UncompressedCachePtr owned_uncompressed_cache; MarkCachePtr owned_mark_cache; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2a659373af9..614748f0ce6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -366,7 +366,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( use_uncompressed_cache = false; MergeTreeReadPoolPtr pool = std::make_shared( - parts, sum_marks_in_parts, sum_marks, data, prewhere_actions, prewhere_column, true, column_names); + parts, data, prewhere_actions, prewhere_column, true, column_names); BlockInputStreams res; From a27057d4b9490396ac4017cb94821e91eafe5de9 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 8 Jul 2015 20:59:44 +0300 Subject: [PATCH 05/50] Merge --- .../DB/IO/CachedCompressedReadBuffer.h | 15 +++-- .../DB/IO/CompressedReadBufferFromFile.h | 8 ++- .../DB/Storages/MergeTree/MergeTreeReader.h | 60 +++++++++++++------ .../MergeTreeThreadBlockInputStream.h | 33 +++++----- dbms/src/IO/ReadBufferAIO.cpp | 8 +-- 5 files changed, 80 insertions(+), 44 deletions(-) diff --git a/dbms/include/DB/IO/CachedCompressedReadBuffer.h b/dbms/include/DB/IO/CachedCompressedReadBuffer.h index 7d71a8ef531..d4fcebeb148 100644 --- a/dbms/include/DB/IO/CachedCompressedReadBuffer.h +++ b/dbms/include/DB/IO/CachedCompressedReadBuffer.h @@ -19,6 +19,7 @@ class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadB private: const std::string path; UncompressedCache * cache; + Memory * memory; size_t buf_size; size_t estimated_size; size_t aio_threshold; @@ -34,7 +35,10 @@ private: { if (!file_in) { - file_in = createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size); + if (memory) + memory->resize(buf_size); + file_in = createReadBufferFromFileBase( + path, estimated_size, aio_threshold, buf_size, -1, memory ? &(*memory)[0] : nullptr); compressed_in = &*file_in; } } @@ -82,10 +86,11 @@ private: } public: - 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) + CachedCompressedReadBuffer( + const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, + size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, Memory * memory = nullptr) + : ReadBuffer(nullptr, 0), path(path_), cache(cache_), memory{memory}, buf_size(buf_size_), + estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0) { } diff --git a/dbms/include/DB/IO/CompressedReadBufferFromFile.h b/dbms/include/DB/IO/CompressedReadBufferFromFile.h index 74af29961e8..3f7a1770a57 100644 --- a/dbms/include/DB/IO/CompressedReadBufferFromFile.h +++ b/dbms/include/DB/IO/CompressedReadBufferFromFile.h @@ -42,10 +42,12 @@ private: } public: - CompressedReadBufferFromFile(const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) + CompressedReadBufferFromFile( + const std::string & path, size_t estimated_size, size_t aio_threshold, + size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr) : BufferWithOwnMemory(0), - p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size)), - file_in(*p_file_in) + p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size, -1, existing_memory)), + file_in(*p_file_in) { compressed_in = &file_in; } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index 9c799c1828b..8f0663fec57 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -32,11 +32,25 @@ public: 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_), - 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_) + : uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_), storage(storage_), + aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_) { + reconf(path_, data_part, columns_, all_mark_ranges); + } + + void reconf( + const String & path, const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns, + const MarkRanges & all_mark_ranges) + { + this->path = path; + this->data_part = data_part; + this->part_name = data_part->name; + this->columns = columns; + this->all_mark_ranges = all_mark_ranges; + this->streams.clear(); + + /// @todo sort buffers using capacity, find best match for Stream. + try { if (!Poco::File(path).exists()) @@ -138,12 +152,11 @@ public: fillMissingColumnsImpl(res, ordered_names, true); } - const MergeTreeData::DataPartPtr & getDataPart() const { return data_part; } - private: struct Stream { MarkCache::MappedPtr marks; + Memory & memory; ReadBuffer * data_buffer; std::unique_ptr cached_buffer; std::unique_ptr non_cached_buffer; @@ -153,9 +166,10 @@ private: /// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины. double avg_value_size_hint = 0; - 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_) + 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, Memory & memory) + : memory(memory), path_prefix(path_prefix_) { loadMarks(mark_cache); size_t max_mark_range = 0; @@ -204,16 +218,21 @@ private: } } + if (aio_threshold == 0 || estimated_size < aio_threshold) + memory.resize(buffer_size); + else + memory.resize(2 * (buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE)); + if (uncompressed_cache) { cached_buffer = std::make_unique( - path_prefix + ".bin", uncompressed_cache, estimated_size, aio_threshold, buffer_size); + path_prefix + ".bin", uncompressed_cache, estimated_size, aio_threshold, buffer_size, &memory); data_buffer = cached_buffer.get(); } else { non_cached_buffer = std::make_unique( - path_prefix + ".bin", estimated_size, aio_threshold, buffer_size); + path_prefix + ".bin", estimated_size, aio_threshold, buffer_size, &memory[0]); data_buffer = non_cached_buffer.get(); } } @@ -276,9 +295,10 @@ private: typedef std::map > FileStreams; String path; - const MergeTreeData::DataPartPtr & data_part; + MergeTreeData::DataPartPtr data_part; String part_name; FileStreams streams; + std::vector> buffers; /// Запрашиваемые столбцы. NamesAndTypesList columns; @@ -287,7 +307,7 @@ private: MarkCache * mark_cache; MergeTreeData & storage; - const MarkRanges & all_mark_ranges; + MarkRanges all_mark_ranges; size_t aio_threshold; size_t max_read_buffer_size; @@ -301,6 +321,10 @@ private: if (!Poco::File(path + escaped_column_name + ".bin").exists()) return; + const auto buffer_idx = streams.size(); + if (buffer_idx >= buffers.size()) + buffers.push_back(std::make_unique(0, DEFAULT_AIO_FILE_BLOCK_SIZE)); + /// Для массивов используются отдельные потоки для размеров. if (const DataTypeArray * type_arr = typeid_cast(&type)) { @@ -310,14 +334,16 @@ private: + ARRAY_SIZES_COLUMN_NAME_SUFFIX + toString(level); if (!streams.count(size_name)) - streams.emplace(size_name, std::unique_ptr(new Stream( - path + escaped_size_name, uncompressed_cache, mark_cache, all_mark_ranges, aio_threshold, max_read_buffer_size))); + streams.emplace(size_name, std::make_unique( + path + escaped_size_name, uncompressed_cache, mark_cache, + all_mark_ranges, aio_threshold, max_read_buffer_size, *buffers[buffer_idx])); 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, aio_threshold, max_read_buffer_size)); + streams.emplace(name, std::make_unique( + path + escaped_column_name, uncompressed_cache, mark_cache, + all_mark_ranges, aio_threshold, max_read_buffer_size, *buffers[buffer_idx])); } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 9c403bc673d..1f31e977d32 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -88,27 +88,31 @@ private: if (!task) return false; - if (!initialized) + const auto path = storage.getFullPath() + task->data_part->name + '/'; + + if (!reader) { if (use_uncompressed_cache) owned_uncompressed_cache = storage.context.getUncompressedCache(); owned_mark_cache = storage.context.getMarkCache(); - initialized = true; + reader = std::make_unique( + path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), + storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); + + if (prewhere_actions) + pre_reader = std::make_unique( + path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), + owned_mark_cache.get(), storage, task->mark_ranges, min_bytes_to_use_direct_io, + max_read_buffer_size); + } + else + { + reader->reconf(path, task->data_part, task->columns, task->mark_ranges); + if (prewhere_actions) + pre_reader->reconf(path, task->data_part, task->pre_columns, task->mark_ranges); } - - const auto path = storage.getFullPath() + task->data_part->name + '/'; - - reader = std::make_unique( - path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), - storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); - - if (prewhere_actions) - pre_reader = std::make_unique( - path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), - owned_mark_cache.get(), storage, task->mark_ranges, min_bytes_to_use_direct_io, - max_read_buffer_size); return true; } @@ -318,7 +322,6 @@ private: using MergeTreeReaderPtr = std::unique_ptr; - bool initialized{false}; UncompressedCachePtr owned_uncompressed_cache; MarkCachePtr owned_mark_cache; diff --git a/dbms/src/IO/ReadBufferAIO.cpp b/dbms/src/IO/ReadBufferAIO.cpp index 9bbb1efbf56..97dce3cfe5d 100644 --- a/dbms/src/IO/ReadBufferAIO.cpp +++ b/dbms/src/IO/ReadBufferAIO.cpp @@ -11,11 +11,11 @@ namespace DB /// Примечание: выделяется дополнительная страница, которая содежрит те данные, которые /// не влезают в основной буфер. -ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, - char * existing_memory_) +ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, char * existing_memory_) : ReadBufferFromFileBase(buffer_size_ + DEFAULT_AIO_FILE_BLOCK_SIZE, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE), - fill_buffer(BufferWithOwnMemory(this->memory.size(), nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)), - filename(filename_) + fill_buffer(BufferWithOwnMemory(internalBuffer().size(), existing_memory_ + internalBuffer().size(), + DEFAULT_AIO_FILE_BLOCK_SIZE)), + filename(filename_) { ProfileEvents::increment(ProfileEvents::FileOpen); From defba956d93c090637a4d2598333992edeffbd22 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 23 Jul 2015 16:11:27 +0300 Subject: [PATCH 06/50] Merge --- .../DB/Storages/MergeTree/MergeTreeReadPool.h | 14 +++++++++++--- .../MergeTree/MergeTreeThreadBlockInputStream.h | 6 ++++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 +++-- 3 files changed, 18 insertions(+), 7 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index a522aca3879..ecfe7511489 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -36,11 +36,13 @@ using MergeTreeReadTaskPtr = std::unique_ptr; class MergeTreeReadPool { + std::size_t threads; public: MergeTreeReadPool( + const std::size_t threads, const RangesInDataParts & parts, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column_name, const bool check_columns, const Names & column_names) - : parts{parts}, data{data}, column_names{column_names} + : threads{threads}, parts{parts}, data{data}, column_names{column_names} { fillPerPartInfo(prewhere_actions, prewhere_column_name, check_columns); } @@ -48,15 +50,17 @@ public: MergeTreeReadPool(const MergeTreeReadPool &) = delete; MergeTreeReadPool & operator=(const MergeTreeReadPool &) = delete; - MergeTreeReadTaskPtr getTask(const std::size_t min_marks_to_read) + MergeTreeReadTaskPtr getTask(const std::size_t min_marks_to_read, const std::size_t thread) { const std::lock_guard lock{mutex}; if (remaining_part_indices.empty()) return nullptr; + const auto idx = remaining_part_indices.size() - (1 + remaining_part_indices.size() * thread / threads); /// find a part which has marks remaining - const auto part_id = remaining_part_indices.back(); +// const auto part_id = remaining_part_indices.back(); + const auto part_id = remaining_part_indices[idx]; auto & part = parts[part_id]; const auto & column_name_set = per_part_column_name_set[part_id]; @@ -87,6 +91,7 @@ public: marks_in_part -= marks_to_get_from_range; + std::swap(remaining_part_indices[idx], remaining_part_indices.back()); remaining_part_indices.pop_back(); } else @@ -109,7 +114,10 @@ public: } if (0 == marks_in_part) + { + std::swap(remaining_part_indices[idx], remaining_part_indices.back()); remaining_part_indices.pop_back(); + } } return std::make_unique( diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 1f31e977d32..6436619a4ac 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -13,13 +13,15 @@ namespace DB class MergeTreeThreadBlockInputStream : public IProfilingBlockInputStream { + std::size_t thread; public: MergeTreeThreadBlockInputStream( + const std::size_t thread, const MergeTreeReadPoolPtr & pool, const std::size_t min_marks_to_read, const std::size_t block_size, MergeTreeData & storage, const bool use_uncompressed_cache, const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column, const std::size_t min_bytes_to_use_direct_io, const std::size_t max_read_buffer_size, const Names & virt_column_names) - : pool{pool}, min_marks_to_read{min_marks_to_read}, block_size{block_size}, storage{storage}, + : thread{thread}, pool{pool}, min_marks_to_read{min_marks_to_read}, block_size{block_size}, storage{storage}, use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions}, prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{min_bytes_to_use_direct_io}, max_read_buffer_size{max_read_buffer_size}, virt_column_names{virt_column_names}, @@ -83,7 +85,7 @@ protected: private: bool getNewTask() { - task = pool->getTask(min_marks_to_read); + task = pool->getTask(min_marks_to_read, thread); if (!task) return false; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 614748f0ce6..0918b69fb35 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -366,7 +366,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( use_uncompressed_cache = false; MergeTreeReadPoolPtr pool = std::make_shared( - parts, data, prewhere_actions, prewhere_column, true, column_names); + threads, parts, data, prewhere_actions, prewhere_column, true, column_names); BlockInputStreams res; @@ -457,7 +457,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( { for (std::size_t i = 0; i < threads; ++i) res.emplace_back(new MergeTreeThreadBlockInputStream{ - pool, min_marks_for_concurrent_read, max_block_size, data, use_uncompressed_cache, prewhere_actions, + i, pool, min_marks_for_concurrent_read, max_block_size, data, use_uncompressed_cache, prewhere_actions, prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns }); @@ -523,6 +523,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal } BlockInputStreams res; + /// @todo parts.size() == 1 ? Filter : Collapse if (to_collapse.size() == 1) res.push_back(new FilterBlockInputStream(new ExpressionBlockInputStream(to_collapse[0], sign_filter_expression), sign_filter_column)); else if (to_collapse.size() > 1) From 9a54c18a8db91997260bc7dcc91d5de5ad6bb67e Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 23 Jul 2015 16:11:27 +0300 Subject: [PATCH 07/50] Merge --- dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index ecfe7511489..f252f48f918 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -99,7 +99,9 @@ public: /// Цикл по отрезкам куска. while (need_marks > 0 && !part.ranges.empty()) { - auto & range = part.ranges.back(); + const auto idx = part.ranges.size() - (1 + part.ranges.size() * thread / threads); +// auto & range = part.ranges.back(); + auto & range = part.ranges[idx]; const std::size_t marks_in_range = range.end - range.begin; const std::size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); @@ -107,7 +109,10 @@ public: ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); range.begin += marks_to_get_from_range; if (range.begin == range.end) + { + std::swap(range, part.ranges.back()); part.ranges.pop_back(); + } marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; From 28d333adff4e512a3be00e910fe2c1b04cd208f7 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 23 Jul 2015 20:54:07 +0300 Subject: [PATCH 08/50] Merge --- .../DB/Storages/MergeTree/MergeTreeReadPool.h | 181 +++++++++++++----- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +- 2 files changed, 140 insertions(+), 44 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index f252f48f918..07a69193fcc 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -36,15 +36,15 @@ using MergeTreeReadTaskPtr = std::unique_ptr; class MergeTreeReadPool { - std::size_t threads; public: MergeTreeReadPool( - const std::size_t threads, - const RangesInDataParts & parts, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions, + const std::size_t threads, const std::size_t sum_marks, const std::size_t min_marks_for_concurrent_read, + RangesInDataParts parts, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column_name, const bool check_columns, const Names & column_names) - : threads{threads}, parts{parts}, data{data}, column_names{column_names} + : data{data}, column_names{column_names} { - fillPerPartInfo(prewhere_actions, prewhere_column_name, check_columns); + const auto per_part_sum_marks = fillPerPartInfo(parts, prewhere_actions, prewhere_column_name, check_columns); + fillPerThreadInfo(threads, sum_marks, per_part_sum_marks, parts, min_marks_for_concurrent_read); } MergeTreeReadPool(const MergeTreeReadPool &) = delete; @@ -54,20 +54,18 @@ public: { const std::lock_guard lock{mutex}; - if (remaining_part_indices.empty()) + if (remaining_thread_tasks.empty()) return nullptr; - const auto idx = remaining_part_indices.size() - (1 + remaining_part_indices.size() * thread / threads); - /// find a part which has marks remaining -// const auto part_id = remaining_part_indices.back(); - const auto part_id = remaining_part_indices[idx]; + const auto thread_idx = !threads_tasks[thread].sum_marks_in_parts.empty() ? thread : + *std::begin(remaining_thread_tasks); + auto & thread_tasks = threads_tasks[thread_idx]; - auto & part = parts[part_id]; - const auto & column_name_set = per_part_column_name_set[part_id]; - const auto & columns = per_part_columns[part_id]; - const auto & pre_columns = per_part_pre_columns[part_id]; - const auto remove_prewhere_column = per_part_remove_prewhere_column[part_id]; - auto & marks_in_part = per_part_sum_marks[part_id]; + auto & thread_task = thread_tasks.parts_and_ranges.back(); + const auto part_idx = thread_task.part_idx; + + auto & part = parts[part_idx]; + auto & marks_in_part = thread_tasks.sum_marks_in_parts.back(); /// Берём весь кусок, если он достаточно мал auto need_marks = std::min(marks_in_part, min_marks_to_read); @@ -84,24 +82,27 @@ public: { const auto marks_to_get_from_range = marks_in_part; + /// @todo fix double reverse /// Восстановим порядок отрезков. - std::reverse(part.ranges.begin(), part.ranges.end()); + std::reverse(thread_task.ranges.begin(), thread_task.ranges.end()); - ranges_to_get_from_part = part.ranges; + ranges_to_get_from_part = thread_task.ranges; marks_in_part -= marks_to_get_from_range; - std::swap(remaining_part_indices[idx], remaining_part_indices.back()); - remaining_part_indices.pop_back(); + thread_tasks.parts_and_ranges.pop_back(); + thread_tasks.sum_marks_in_parts.pop_back(); + + if (thread_tasks.sum_marks_in_parts.empty()) + remaining_thread_tasks.erase(thread_idx); } else { /// Цикл по отрезкам куска. - while (need_marks > 0 && !part.ranges.empty()) + while (need_marks > 0 && !thread_task.ranges.empty()) { - const auto idx = part.ranges.size() - (1 + part.ranges.size() * thread / threads); -// auto & range = part.ranges.back(); - auto & range = part.ranges[idx]; + /// @todo fix double reverse + auto & range = thread_task.ranges.back(); const std::size_t marks_in_range = range.end - range.begin; const std::size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); @@ -110,31 +111,27 @@ public: range.begin += marks_to_get_from_range; if (range.begin == range.end) { - std::swap(range, part.ranges.back()); - part.ranges.pop_back(); + std::swap(range, thread_task.ranges.back()); + thread_task.ranges.pop_back(); } marks_in_part -= marks_to_get_from_range; need_marks -= marks_to_get_from_range; } - - if (0 == marks_in_part) - { - std::swap(remaining_part_indices[idx], remaining_part_indices.back()); - remaining_part_indices.pop_back(); - } } return std::make_unique( - part.data_part, ranges_to_get_from_part, part.part_index_in_query, column_names, column_name_set, columns, - pre_columns, remove_prewhere_column, per_part_should_reorder[part_id]); + part.data_part, thread_task.ranges, part.part_index_in_query, column_names, + per_part_column_name_set[part_idx], per_part_columns[part_idx], per_part_pre_columns[part_idx], + per_part_remove_prewhere_column[part_idx], per_part_should_reorder[part_idx]); } public: - void fillPerPartInfo( - const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column_name, const bool check_columns) + std::vector fillPerPartInfo( + RangesInDataParts & parts, const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column_name, + const bool check_columns) { - remaining_part_indices.reserve(parts.size()); + std::vector per_part_sum_marks; for (const auto i : ext::range(0, parts.size())) { @@ -150,9 +147,6 @@ public: per_part_sum_marks.push_back(sum_marks); - if (0 != sum_marks) - remaining_part_indices.push_back(i); - per_part_columns_lock.push_back(std::make_unique( part.data_part->columns_lock)); @@ -217,6 +211,86 @@ public: } per_part_should_reorder.push_back(should_reoder); + + this->parts.push_back({ part.data_part, part.part_index_in_query }); + } + + return per_part_sum_marks; + } + + void fillPerThreadInfo( + const std::size_t threads, const std::size_t sum_marks, std::vector per_part_sum_marks, + RangesInDataParts & parts, const std::size_t min_marks_for_concurrent_read) + { + threads_tasks.resize(threads); + + const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; + + for (std::size_t i = 0; i < threads && !parts.empty(); ++i) + { + auto need_marks = min_marks_per_thread; + + while (need_marks > 0 && !parts.empty()) + { + const auto part_idx = parts.size() - 1; + RangesInDataPart & part = parts.back(); + size_t & marks_in_part = per_part_sum_marks.back(); + + /// Не будем брать из куска слишком мало строк. + if (marks_in_part >= min_marks_for_concurrent_read && + need_marks < min_marks_for_concurrent_read) + need_marks = min_marks_for_concurrent_read; + + /// Не будем оставлять в куске слишком мало строк. + if (marks_in_part > need_marks && + marks_in_part - need_marks < min_marks_for_concurrent_read) + need_marks = marks_in_part; + + MarkRanges ranges_to_get_from_part; + size_t marks_in_ranges = need_marks; + + /// Возьмем весь кусок, если он достаточно мал. + if (marks_in_part <= need_marks) + { + /// @todo fix double reverse + /// Восстановим порядок отрезков. + std::reverse(part.ranges.begin(), part.ranges.end()); + + ranges_to_get_from_part = part.ranges; + marks_in_ranges = marks_in_part; + + need_marks -= marks_in_part; + parts.pop_back(); + per_part_sum_marks.pop_back(); + } + else + { + /// Цикл по отрезкам куска. + while (need_marks > 0) + { + /// @todo fix double reverse + if (part.ranges.empty()) + throw Exception("Unexpected end of ranges while spreading marks among threads", ErrorCodes::LOGICAL_ERROR); + + MarkRange & range = part.ranges.back(); + + const size_t marks_in_range = range.end - range.begin; + const size_t marks_to_get_from_range = std::min(marks_in_range, need_marks); + + ranges_to_get_from_part.emplace_back(range.begin, range.begin + marks_to_get_from_range); + range.begin += marks_to_get_from_range; + marks_in_part -= marks_to_get_from_range; + need_marks -= marks_to_get_from_range; + if (range.begin == range.end) + part.ranges.pop_back(); + } + } + + threads_tasks[i].parts_and_ranges.push_back({ part_idx, ranges_to_get_from_part }); + threads_tasks[i].sum_marks_in_parts.push_back(marks_in_ranges); + if (marks_in_ranges != 0) + remaining_thread_tasks.insert(i); + } } } @@ -319,9 +393,6 @@ public: } std::vector> per_part_columns_lock; - RangesInDataParts parts; - std::vector per_part_sum_marks; - std::vector remaining_part_indices; MergeTreeData & data; Names column_names; std::vector per_part_column_name_set; @@ -331,6 +402,30 @@ public: std::vector per_part_remove_prewhere_column; std::vector per_part_should_reorder; + struct part_t + { + MergeTreeData::DataPartPtr data_part; + std::size_t part_index_in_query; + }; + + std::vector parts; + + struct thread_task_t + { + struct part_index_and_range_t + { + std::size_t part_idx; + MarkRanges ranges; + }; + + std::vector parts_and_ranges; + std::vector sum_marks_in_parts; + }; + + std::vector threads_tasks; + + std::unordered_set remaining_thread_tasks; + mutable std::mutex mutex; }; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 0918b69fb35..55665e09b25 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -366,7 +366,8 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( use_uncompressed_cache = false; MergeTreeReadPoolPtr pool = std::make_shared( - threads, parts, data, prewhere_actions, prewhere_column, true, column_names); + threads, sum_marks, min_marks_for_concurrent_read, parts, data, prewhere_actions, prewhere_column, true, + column_names); BlockInputStreams res; From 9fea075cb025f39b9c95df35cab69748b3dcb893 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 23 Jul 2015 20:54:07 +0300 Subject: [PATCH 09/50] Merge --- dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h | 2 +- .../DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index 07a69193fcc..d1766c27e60 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -140,7 +140,7 @@ public: /// Посчитаем засечки для каждого куска. size_t sum_marks = 0; /// Пусть отрезки будут перечислены справа налево, чтобы можно было выбрасывать самый левый отрезок с помощью pop_back(). - std::reverse(std::begin(part.ranges), std::end(part.ranges)); +// std::reverse(std::begin(part.ranges), std::end(part.ranges)); for (const auto & range : part.ranges) sum_marks += range.end - range.begin; diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 6436619a4ac..22ac5e9822b 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -74,8 +74,6 @@ protected: * Чтобы при создании многих источников, но одновременном чтении только из нескольких, * буферы не висели в памяти. */ task = {}; - reader = {}; - pre_reader = {}; } } @@ -88,7 +86,11 @@ private: task = pool->getTask(min_marks_to_read, thread); if (!task) + { + reader = {}; + pre_reader = {}; return false; + } const auto path = storage.getFullPath() + task->data_part->name + '/'; From 8b890f1c0667be8c6eb7edefc1169ed05f809a04 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 20 Aug 2015 15:07:29 +0300 Subject: [PATCH 10/50] dbms: add functions bitTestAny, bitTestAll (non-vectorized) and tests [#METR-17245] --- dbms/include/DB/Functions/FunctionsCoding.h | 171 ++++ dbms/src/Functions/FunctionsCoding.cpp | 2 + .../00216_bit_test_function_family.reference | 771 ++++++++++++++++++ .../00216_bit_test_function_family.sql | 115 +++ 4 files changed, 1059 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00216_bit_test_function_family.reference create mode 100644 dbms/tests/queries/0_stateless/00216_bit_test_function_family.sql diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index f282230f70b..b4aaddeca1f 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -1447,6 +1447,7 @@ public: }; } +private: template bool execute( Block & block, const ColumnNumbers & arguments, const size_t result, const IColumn * const value_col_untyped) @@ -1564,4 +1565,174 @@ public: } }; +template +struct FunctionBitTestMany : public IFunction +{ +public: + static constexpr auto && name = Impl::name; + static IFunction * create(const Context &) { return new FunctionBitTestMany; } + + String getName() const override { return name; } + + DataTypePtr getReturnType(const DataTypes & arguments) const override + { + if (arguments.size() < 2) + throw Exception{ + "Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be at least 2.", + ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION + }; + + const auto first_arg = arguments.front().get(); + if (!typeid_cast(first_arg) && !typeid_cast(first_arg) && + !typeid_cast(first_arg) && !typeid_cast(first_arg) && + !typeid_cast(first_arg) && !typeid_cast(first_arg) && + !typeid_cast(first_arg) && !typeid_cast(first_arg)) + throw Exception{ + "Illegal type " + first_arg->getName() + " of first argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + + + for (const auto i : ext::range(1, arguments.size())) + { + const auto pos_arg = arguments[i].get(); + + if (!typeid_cast(pos_arg) && !typeid_cast(pos_arg) && + !typeid_cast(pos_arg) && !typeid_cast(pos_arg)) + throw Exception{ + "Illegal type " + pos_arg->getName() + " of " + toString(i) + " argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT + }; + } + + return new DataTypeUInt8; + } + + void execute(Block & block, const ColumnNumbers & arguments, const size_t result) override + { + const auto value_col = block.getByPosition(arguments.front()).column.get(); + + if (!execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col) && + !execute(block, arguments, result, value_col)) + throw Exception{ + "Illegal column " + value_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + } + +private: + template bool execute( + Block & block, const ColumnNumbers & arguments, const size_t result, + const IColumn * const value_col_untyped) + { + if (const auto value_col = typeid_cast *>(value_col_untyped)) + { + const auto size = value_col->size(); + const auto & mask = createMask(size, block, arguments); + const auto & val = value_col->getData(); + + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; + + auto & out = out_col->getData(); + + for (const auto i : ext::range(0, size)) + out[i] = Impl::combine(val[i], mask[i]); + + return true; + } + else if (const auto value_col = typeid_cast *>(value_col_untyped)) + { + const auto size = value_col->size(); + const auto & mask = createMask(size, block, arguments); + const auto & val = value_col->getData(); + + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; + + auto & out = out_col->getData(); + + for (const auto i : ext::range(0, size)) + out[i] = Impl::combine(val, mask[i]); + + return true; + } + + return false; + } + + template + PODArray createMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments) + { + PODArray mask(size, T{}); + + for (const auto i : ext::range(1, arguments.size())) + addToMask(mask, block.getByPosition(arguments[i]).column.get()); + + return mask; + } + + template + void addToMask(PODArray & mask, const IColumn * const pos_col) + { + if (!addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col) && + !addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col)) + throw Exception{ + "Illegal column " + pos_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + } + + template + bool addToMaskImpl(PODArray & mask, const IColumn * const pos_col_untyped) + { + if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + const auto & pos = pos_col->getData(); + + for (const auto i : ext::range(0, mask.size())) + mask[i] = mask[i] | (1 << pos[i]); + + return true; + } + else if (const auto pos_col = typeid_cast *>(pos_col_untyped)) + { + const auto & pos = pos_col->getData(); + const auto new_mask = 1 << pos; + + for (const auto i : ext::range(0, mask.size())) + mask[i] = mask[i] | new_mask; + + return true; + } + + return false; + } +}; + +struct BitTestAnyImpl +{ + static constexpr auto name = "bitTestAny"; + template static UInt8 combine(const T val, const T mask) { return (val & mask) != 0; } +}; + +struct BitTestAllImpl +{ + static constexpr auto name = "bitTestAll"; + template static UInt8 combine(const T val, const T mask) { return (val & mask) == mask; } +}; + +using FunctionBitTestAny = FunctionBitTestMany; +using FunctionBitTestAll = FunctionBitTestMany; + + } diff --git a/dbms/src/Functions/FunctionsCoding.cpp b/dbms/src/Functions/FunctionsCoding.cpp index a7a76f7ed21..d213f0f6249 100644 --- a/dbms/src/Functions/FunctionsCoding.cpp +++ b/dbms/src/Functions/FunctionsCoding.cpp @@ -16,6 +16,8 @@ void registerFunctionsCoding(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); } } diff --git a/dbms/tests/queries/0_stateless/00216_bit_test_function_family.reference b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.reference new file mode 100644 index 00000000000..54ece010c27 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.reference @@ -0,0 +1,771 @@ +1 1 1 1 +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 +1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 1 +1 0 0 0 0 0 0 0 0 0 0 +1 1 1 0 0 0 0 0 0 0 1 +1 2 2 0 0 0 0 0 0 1 0 +1 3 3 0 0 0 0 0 0 1 1 +1 4 4 0 0 0 0 0 1 0 0 +1 5 5 0 0 0 0 0 1 0 1 +1 6 6 0 0 0 0 0 1 1 0 +1 7 7 0 0 0 0 0 1 1 1 +1 8 8 0 0 0 0 1 0 0 0 +1 9 9 0 0 0 0 1 0 0 1 +1 10 10 0 0 0 0 1 0 1 0 +1 11 11 0 0 0 0 1 0 1 1 +1 12 12 0 0 0 0 1 1 0 0 +1 13 13 0 0 0 0 1 1 0 1 +1 14 14 0 0 0 0 1 1 1 0 +1 15 15 0 0 0 0 1 1 1 1 +1 16 16 0 0 0 1 0 0 0 0 +1 17 17 0 0 0 1 0 0 0 1 +1 18 18 0 0 0 1 0 0 1 0 +1 19 19 0 0 0 1 0 0 1 1 +1 20 20 0 0 0 1 0 1 0 0 +1 21 21 0 0 0 1 0 1 0 1 +1 22 22 0 0 0 1 0 1 1 0 +1 23 23 0 0 0 1 0 1 1 1 +1 24 24 0 0 0 1 1 0 0 0 +1 25 25 0 0 0 1 1 0 0 1 +1 26 26 0 0 0 1 1 0 1 0 +1 27 27 0 0 0 1 1 0 1 1 +1 28 28 0 0 0 1 1 1 0 0 +1 29 29 0 0 0 1 1 1 0 1 +1 30 30 0 0 0 1 1 1 1 0 +1 31 31 0 0 0 1 1 1 1 1 +1 32 32 0 0 1 0 0 0 0 0 +1 33 33 0 0 1 0 0 0 0 1 +1 34 34 0 0 1 0 0 0 1 0 +1 35 35 0 0 1 0 0 0 1 1 +1 36 36 0 0 1 0 0 1 0 0 +1 37 37 0 0 1 0 0 1 0 1 +1 38 38 0 0 1 0 0 1 1 0 +1 39 39 0 0 1 0 0 1 1 1 +1 40 40 0 0 1 0 1 0 0 0 +1 41 41 0 0 1 0 1 0 0 1 +1 42 42 0 0 1 0 1 0 1 0 +1 43 43 0 0 1 0 1 0 1 1 +1 44 44 0 0 1 0 1 1 0 0 +1 45 45 0 0 1 0 1 1 0 1 +1 46 46 0 0 1 0 1 1 1 0 +1 47 47 0 0 1 0 1 1 1 1 +1 48 48 0 0 1 1 0 0 0 0 +1 49 49 0 0 1 1 0 0 0 1 +1 50 50 0 0 1 1 0 0 1 0 +1 51 51 0 0 1 1 0 0 1 1 +1 52 52 0 0 1 1 0 1 0 0 +1 53 53 0 0 1 1 0 1 0 1 +1 54 54 0 0 1 1 0 1 1 0 +1 55 55 0 0 1 1 0 1 1 1 +1 56 56 0 0 1 1 1 0 0 0 +1 57 57 0 0 1 1 1 0 0 1 +1 58 58 0 0 1 1 1 0 1 0 +1 59 59 0 0 1 1 1 0 1 1 +1 60 60 0 0 1 1 1 1 0 0 +1 61 61 0 0 1 1 1 1 0 1 +1 62 62 0 0 1 1 1 1 1 0 +1 63 63 0 0 1 1 1 1 1 1 +1 64 64 0 1 0 0 0 0 0 0 +1 65 65 0 1 0 0 0 0 0 1 +1 66 66 0 1 0 0 0 0 1 0 +1 67 67 0 1 0 0 0 0 1 1 +1 68 68 0 1 0 0 0 1 0 0 +1 69 69 0 1 0 0 0 1 0 1 +1 70 70 0 1 0 0 0 1 1 0 +1 71 71 0 1 0 0 0 1 1 1 +1 72 72 0 1 0 0 1 0 0 0 +1 73 73 0 1 0 0 1 0 0 1 +1 74 74 0 1 0 0 1 0 1 0 +1 75 75 0 1 0 0 1 0 1 1 +1 76 76 0 1 0 0 1 1 0 0 +1 77 77 0 1 0 0 1 1 0 1 +1 78 78 0 1 0 0 1 1 1 0 +1 79 79 0 1 0 0 1 1 1 1 +1 80 80 0 1 0 1 0 0 0 0 +1 81 81 0 1 0 1 0 0 0 1 +1 82 82 0 1 0 1 0 0 1 0 +1 83 83 0 1 0 1 0 0 1 1 +1 84 84 0 1 0 1 0 1 0 0 +1 85 85 0 1 0 1 0 1 0 1 +1 86 86 0 1 0 1 0 1 1 0 +1 87 87 0 1 0 1 0 1 1 1 +1 88 88 0 1 0 1 1 0 0 0 +1 89 89 0 1 0 1 1 0 0 1 +1 90 90 0 1 0 1 1 0 1 0 +1 91 91 0 1 0 1 1 0 1 1 +1 92 92 0 1 0 1 1 1 0 0 +1 93 93 0 1 0 1 1 1 0 1 +1 94 94 0 1 0 1 1 1 1 0 +1 95 95 0 1 0 1 1 1 1 1 +1 96 96 0 1 1 0 0 0 0 0 +1 97 97 0 1 1 0 0 0 0 1 +1 98 98 0 1 1 0 0 0 1 0 +1 99 99 0 1 1 0 0 0 1 1 +1 100 100 0 1 1 0 0 1 0 0 +1 101 101 0 1 1 0 0 1 0 1 +1 102 102 0 1 1 0 0 1 1 0 +1 103 103 0 1 1 0 0 1 1 1 +1 104 104 0 1 1 0 1 0 0 0 +1 105 105 0 1 1 0 1 0 0 1 +1 106 106 0 1 1 0 1 0 1 0 +1 107 107 0 1 1 0 1 0 1 1 +1 108 108 0 1 1 0 1 1 0 0 +1 109 109 0 1 1 0 1 1 0 1 +1 110 110 0 1 1 0 1 1 1 0 +1 111 111 0 1 1 0 1 1 1 1 +1 112 112 0 1 1 1 0 0 0 0 +1 113 113 0 1 1 1 0 0 0 1 +1 114 114 0 1 1 1 0 0 1 0 +1 115 115 0 1 1 1 0 0 1 1 +1 116 116 0 1 1 1 0 1 0 0 +1 117 117 0 1 1 1 0 1 0 1 +1 118 118 0 1 1 1 0 1 1 0 +1 119 119 0 1 1 1 0 1 1 1 +1 120 120 0 1 1 1 1 0 0 0 +1 121 121 0 1 1 1 1 0 0 1 +1 122 122 0 1 1 1 1 0 1 0 +1 123 123 0 1 1 1 1 0 1 1 +1 124 124 0 1 1 1 1 1 0 0 +1 125 125 0 1 1 1 1 1 0 1 +1 126 126 0 1 1 1 1 1 1 0 +1 127 127 0 1 1 1 1 1 1 1 +1 128 128 1 0 0 0 0 0 0 0 +1 129 129 1 0 0 0 0 0 0 1 +1 130 130 1 0 0 0 0 0 1 0 +1 131 131 1 0 0 0 0 0 1 1 +1 132 132 1 0 0 0 0 1 0 0 +1 133 133 1 0 0 0 0 1 0 1 +1 134 134 1 0 0 0 0 1 1 0 +1 135 135 1 0 0 0 0 1 1 1 +1 136 136 1 0 0 0 1 0 0 0 +1 137 137 1 0 0 0 1 0 0 1 +1 138 138 1 0 0 0 1 0 1 0 +1 139 139 1 0 0 0 1 0 1 1 +1 140 140 1 0 0 0 1 1 0 0 +1 141 141 1 0 0 0 1 1 0 1 +1 142 142 1 0 0 0 1 1 1 0 +1 143 143 1 0 0 0 1 1 1 1 +1 144 144 1 0 0 1 0 0 0 0 +1 145 145 1 0 0 1 0 0 0 1 +1 146 146 1 0 0 1 0 0 1 0 +1 147 147 1 0 0 1 0 0 1 1 +1 148 148 1 0 0 1 0 1 0 0 +1 149 149 1 0 0 1 0 1 0 1 +1 150 150 1 0 0 1 0 1 1 0 +1 151 151 1 0 0 1 0 1 1 1 +1 152 152 1 0 0 1 1 0 0 0 +1 153 153 1 0 0 1 1 0 0 1 +1 154 154 1 0 0 1 1 0 1 0 +1 155 155 1 0 0 1 1 0 1 1 +1 156 156 1 0 0 1 1 1 0 0 +1 157 157 1 0 0 1 1 1 0 1 +1 158 158 1 0 0 1 1 1 1 0 +1 159 159 1 0 0 1 1 1 1 1 +1 160 160 1 0 1 0 0 0 0 0 +1 161 161 1 0 1 0 0 0 0 1 +1 162 162 1 0 1 0 0 0 1 0 +1 163 163 1 0 1 0 0 0 1 1 +1 164 164 1 0 1 0 0 1 0 0 +1 165 165 1 0 1 0 0 1 0 1 +1 166 166 1 0 1 0 0 1 1 0 +1 167 167 1 0 1 0 0 1 1 1 +1 168 168 1 0 1 0 1 0 0 0 +1 169 169 1 0 1 0 1 0 0 1 +1 170 170 1 0 1 0 1 0 1 0 +1 171 171 1 0 1 0 1 0 1 1 +1 172 172 1 0 1 0 1 1 0 0 +1 173 173 1 0 1 0 1 1 0 1 +1 174 174 1 0 1 0 1 1 1 0 +1 175 175 1 0 1 0 1 1 1 1 +1 176 176 1 0 1 1 0 0 0 0 +1 177 177 1 0 1 1 0 0 0 1 +1 178 178 1 0 1 1 0 0 1 0 +1 179 179 1 0 1 1 0 0 1 1 +1 180 180 1 0 1 1 0 1 0 0 +1 181 181 1 0 1 1 0 1 0 1 +1 182 182 1 0 1 1 0 1 1 0 +1 183 183 1 0 1 1 0 1 1 1 +1 184 184 1 0 1 1 1 0 0 0 +1 185 185 1 0 1 1 1 0 0 1 +1 186 186 1 0 1 1 1 0 1 0 +1 187 187 1 0 1 1 1 0 1 1 +1 188 188 1 0 1 1 1 1 0 0 +1 189 189 1 0 1 1 1 1 0 1 +1 190 190 1 0 1 1 1 1 1 0 +1 191 191 1 0 1 1 1 1 1 1 +1 192 192 1 1 0 0 0 0 0 0 +1 193 193 1 1 0 0 0 0 0 1 +1 194 194 1 1 0 0 0 0 1 0 +1 195 195 1 1 0 0 0 0 1 1 +1 196 196 1 1 0 0 0 1 0 0 +1 197 197 1 1 0 0 0 1 0 1 +1 198 198 1 1 0 0 0 1 1 0 +1 199 199 1 1 0 0 0 1 1 1 +1 200 200 1 1 0 0 1 0 0 0 +1 201 201 1 1 0 0 1 0 0 1 +1 202 202 1 1 0 0 1 0 1 0 +1 203 203 1 1 0 0 1 0 1 1 +1 204 204 1 1 0 0 1 1 0 0 +1 205 205 1 1 0 0 1 1 0 1 +1 206 206 1 1 0 0 1 1 1 0 +1 207 207 1 1 0 0 1 1 1 1 +1 208 208 1 1 0 1 0 0 0 0 +1 209 209 1 1 0 1 0 0 0 1 +1 210 210 1 1 0 1 0 0 1 0 +1 211 211 1 1 0 1 0 0 1 1 +1 212 212 1 1 0 1 0 1 0 0 +1 213 213 1 1 0 1 0 1 0 1 +1 214 214 1 1 0 1 0 1 1 0 +1 215 215 1 1 0 1 0 1 1 1 +1 216 216 1 1 0 1 1 0 0 0 +1 217 217 1 1 0 1 1 0 0 1 +1 218 218 1 1 0 1 1 0 1 0 +1 219 219 1 1 0 1 1 0 1 1 +1 220 220 1 1 0 1 1 1 0 0 +1 221 221 1 1 0 1 1 1 0 1 +1 222 222 1 1 0 1 1 1 1 0 +1 223 223 1 1 0 1 1 1 1 1 +1 224 224 1 1 1 0 0 0 0 0 +1 225 225 1 1 1 0 0 0 0 1 +1 226 226 1 1 1 0 0 0 1 0 +1 227 227 1 1 1 0 0 0 1 1 +1 228 228 1 1 1 0 0 1 0 0 +1 229 229 1 1 1 0 0 1 0 1 +1 230 230 1 1 1 0 0 1 1 0 +1 231 231 1 1 1 0 0 1 1 1 +1 232 232 1 1 1 0 1 0 0 0 +1 233 233 1 1 1 0 1 0 0 1 +1 234 234 1 1 1 0 1 0 1 0 +1 235 235 1 1 1 0 1 0 1 1 +1 236 236 1 1 1 0 1 1 0 0 +1 237 237 1 1 1 0 1 1 0 1 +1 238 238 1 1 1 0 1 1 1 0 +1 239 239 1 1 1 0 1 1 1 1 +1 240 240 1 1 1 1 0 0 0 0 +1 241 241 1 1 1 1 0 0 0 1 +1 242 242 1 1 1 1 0 0 1 0 +1 243 243 1 1 1 1 0 0 1 1 +1 244 244 1 1 1 1 0 1 0 0 +1 245 245 1 1 1 1 0 1 0 1 +1 246 246 1 1 1 1 0 1 1 0 +1 247 247 1 1 1 1 0 1 1 1 +1 248 248 1 1 1 1 1 0 0 0 +1 249 249 1 1 1 1 1 0 0 1 +1 250 250 1 1 1 1 1 0 1 0 +1 251 251 1 1 1 1 1 0 1 1 +1 252 252 1 1 1 1 1 1 0 0 +1 253 253 1 1 1 1 1 1 0 1 +1 254 254 1 1 1 1 1 1 1 0 +1 255 255 1 1 1 1 1 1 1 1 +1 0 0 0 0 0 0 0 0 0 0 +1 1 1 0 0 0 0 0 0 0 1 +1 2 2 0 0 0 0 0 0 1 0 +1 3 3 0 0 0 0 0 0 1 1 +1 4 4 0 0 0 0 0 1 0 0 +1 5 5 0 0 0 0 0 1 0 1 +1 6 6 0 0 0 0 0 1 1 0 +1 7 7 0 0 0 0 0 1 1 1 +1 8 8 0 0 0 0 1 0 0 0 +1 9 9 0 0 0 0 1 0 0 1 +1 10 10 0 0 0 0 1 0 1 0 +1 11 11 0 0 0 0 1 0 1 1 +1 12 12 0 0 0 0 1 1 0 0 +1 13 13 0 0 0 0 1 1 0 1 +1 14 14 0 0 0 0 1 1 1 0 +1 15 15 0 0 0 0 1 1 1 1 +1 16 16 0 0 0 1 0 0 0 0 +1 17 17 0 0 0 1 0 0 0 1 +1 18 18 0 0 0 1 0 0 1 0 +1 19 19 0 0 0 1 0 0 1 1 +1 20 20 0 0 0 1 0 1 0 0 +1 21 21 0 0 0 1 0 1 0 1 +1 22 22 0 0 0 1 0 1 1 0 +1 23 23 0 0 0 1 0 1 1 1 +1 24 24 0 0 0 1 1 0 0 0 +1 25 25 0 0 0 1 1 0 0 1 +1 26 26 0 0 0 1 1 0 1 0 +1 27 27 0 0 0 1 1 0 1 1 +1 28 28 0 0 0 1 1 1 0 0 +1 29 29 0 0 0 1 1 1 0 1 +1 30 30 0 0 0 1 1 1 1 0 +1 31 31 0 0 0 1 1 1 1 1 +1 32 32 0 0 1 0 0 0 0 0 +1 33 33 0 0 1 0 0 0 0 1 +1 34 34 0 0 1 0 0 0 1 0 +1 35 35 0 0 1 0 0 0 1 1 +1 36 36 0 0 1 0 0 1 0 0 +1 37 37 0 0 1 0 0 1 0 1 +1 38 38 0 0 1 0 0 1 1 0 +1 39 39 0 0 1 0 0 1 1 1 +1 40 40 0 0 1 0 1 0 0 0 +1 41 41 0 0 1 0 1 0 0 1 +1 42 42 0 0 1 0 1 0 1 0 +1 43 43 0 0 1 0 1 0 1 1 +1 44 44 0 0 1 0 1 1 0 0 +1 45 45 0 0 1 0 1 1 0 1 +1 46 46 0 0 1 0 1 1 1 0 +1 47 47 0 0 1 0 1 1 1 1 +1 48 48 0 0 1 1 0 0 0 0 +1 49 49 0 0 1 1 0 0 0 1 +1 50 50 0 0 1 1 0 0 1 0 +1 51 51 0 0 1 1 0 0 1 1 +1 52 52 0 0 1 1 0 1 0 0 +1 53 53 0 0 1 1 0 1 0 1 +1 54 54 0 0 1 1 0 1 1 0 +1 55 55 0 0 1 1 0 1 1 1 +1 56 56 0 0 1 1 1 0 0 0 +1 57 57 0 0 1 1 1 0 0 1 +1 58 58 0 0 1 1 1 0 1 0 +1 59 59 0 0 1 1 1 0 1 1 +1 60 60 0 0 1 1 1 1 0 0 +1 61 61 0 0 1 1 1 1 0 1 +1 62 62 0 0 1 1 1 1 1 0 +1 63 63 0 0 1 1 1 1 1 1 +1 64 64 0 1 0 0 0 0 0 0 +1 65 65 0 1 0 0 0 0 0 1 +1 66 66 0 1 0 0 0 0 1 0 +1 67 67 0 1 0 0 0 0 1 1 +1 68 68 0 1 0 0 0 1 0 0 +1 69 69 0 1 0 0 0 1 0 1 +1 70 70 0 1 0 0 0 1 1 0 +1 71 71 0 1 0 0 0 1 1 1 +1 72 72 0 1 0 0 1 0 0 0 +1 73 73 0 1 0 0 1 0 0 1 +1 74 74 0 1 0 0 1 0 1 0 +1 75 75 0 1 0 0 1 0 1 1 +1 76 76 0 1 0 0 1 1 0 0 +1 77 77 0 1 0 0 1 1 0 1 +1 78 78 0 1 0 0 1 1 1 0 +1 79 79 0 1 0 0 1 1 1 1 +1 80 80 0 1 0 1 0 0 0 0 +1 81 81 0 1 0 1 0 0 0 1 +1 82 82 0 1 0 1 0 0 1 0 +1 83 83 0 1 0 1 0 0 1 1 +1 84 84 0 1 0 1 0 1 0 0 +1 85 85 0 1 0 1 0 1 0 1 +1 86 86 0 1 0 1 0 1 1 0 +1 87 87 0 1 0 1 0 1 1 1 +1 88 88 0 1 0 1 1 0 0 0 +1 89 89 0 1 0 1 1 0 0 1 +1 90 90 0 1 0 1 1 0 1 0 +1 91 91 0 1 0 1 1 0 1 1 +1 92 92 0 1 0 1 1 1 0 0 +1 93 93 0 1 0 1 1 1 0 1 +1 94 94 0 1 0 1 1 1 1 0 +1 95 95 0 1 0 1 1 1 1 1 +1 96 96 0 1 1 0 0 0 0 0 +1 97 97 0 1 1 0 0 0 0 1 +1 98 98 0 1 1 0 0 0 1 0 +1 99 99 0 1 1 0 0 0 1 1 +1 100 100 0 1 1 0 0 1 0 0 +1 101 101 0 1 1 0 0 1 0 1 +1 102 102 0 1 1 0 0 1 1 0 +1 103 103 0 1 1 0 0 1 1 1 +1 104 104 0 1 1 0 1 0 0 0 +1 105 105 0 1 1 0 1 0 0 1 +1 106 106 0 1 1 0 1 0 1 0 +1 107 107 0 1 1 0 1 0 1 1 +1 108 108 0 1 1 0 1 1 0 0 +1 109 109 0 1 1 0 1 1 0 1 +1 110 110 0 1 1 0 1 1 1 0 +1 111 111 0 1 1 0 1 1 1 1 +1 112 112 0 1 1 1 0 0 0 0 +1 113 113 0 1 1 1 0 0 0 1 +1 114 114 0 1 1 1 0 0 1 0 +1 115 115 0 1 1 1 0 0 1 1 +1 116 116 0 1 1 1 0 1 0 0 +1 117 117 0 1 1 1 0 1 0 1 +1 118 118 0 1 1 1 0 1 1 0 +1 119 119 0 1 1 1 0 1 1 1 +1 120 120 0 1 1 1 1 0 0 0 +1 121 121 0 1 1 1 1 0 0 1 +1 122 122 0 1 1 1 1 0 1 0 +1 123 123 0 1 1 1 1 0 1 1 +1 124 124 0 1 1 1 1 1 0 0 +1 125 125 0 1 1 1 1 1 0 1 +1 126 126 0 1 1 1 1 1 1 0 +1 127 127 0 1 1 1 1 1 1 1 +1 128 128 1 0 0 0 0 0 0 0 +1 129 129 1 0 0 0 0 0 0 1 +1 130 130 1 0 0 0 0 0 1 0 +1 131 131 1 0 0 0 0 0 1 1 +1 132 132 1 0 0 0 0 1 0 0 +1 133 133 1 0 0 0 0 1 0 1 +1 134 134 1 0 0 0 0 1 1 0 +1 135 135 1 0 0 0 0 1 1 1 +1 136 136 1 0 0 0 1 0 0 0 +1 137 137 1 0 0 0 1 0 0 1 +1 138 138 1 0 0 0 1 0 1 0 +1 139 139 1 0 0 0 1 0 1 1 +1 140 140 1 0 0 0 1 1 0 0 +1 141 141 1 0 0 0 1 1 0 1 +1 142 142 1 0 0 0 1 1 1 0 +1 143 143 1 0 0 0 1 1 1 1 +1 144 144 1 0 0 1 0 0 0 0 +1 145 145 1 0 0 1 0 0 0 1 +1 146 146 1 0 0 1 0 0 1 0 +1 147 147 1 0 0 1 0 0 1 1 +1 148 148 1 0 0 1 0 1 0 0 +1 149 149 1 0 0 1 0 1 0 1 +1 150 150 1 0 0 1 0 1 1 0 +1 151 151 1 0 0 1 0 1 1 1 +1 152 152 1 0 0 1 1 0 0 0 +1 153 153 1 0 0 1 1 0 0 1 +1 154 154 1 0 0 1 1 0 1 0 +1 155 155 1 0 0 1 1 0 1 1 +1 156 156 1 0 0 1 1 1 0 0 +1 157 157 1 0 0 1 1 1 0 1 +1 158 158 1 0 0 1 1 1 1 0 +1 159 159 1 0 0 1 1 1 1 1 +1 160 160 1 0 1 0 0 0 0 0 +1 161 161 1 0 1 0 0 0 0 1 +1 162 162 1 0 1 0 0 0 1 0 +1 163 163 1 0 1 0 0 0 1 1 +1 164 164 1 0 1 0 0 1 0 0 +1 165 165 1 0 1 0 0 1 0 1 +1 166 166 1 0 1 0 0 1 1 0 +1 167 167 1 0 1 0 0 1 1 1 +1 168 168 1 0 1 0 1 0 0 0 +1 169 169 1 0 1 0 1 0 0 1 +1 170 170 1 0 1 0 1 0 1 0 +1 171 171 1 0 1 0 1 0 1 1 +1 172 172 1 0 1 0 1 1 0 0 +1 173 173 1 0 1 0 1 1 0 1 +1 174 174 1 0 1 0 1 1 1 0 +1 175 175 1 0 1 0 1 1 1 1 +1 176 176 1 0 1 1 0 0 0 0 +1 177 177 1 0 1 1 0 0 0 1 +1 178 178 1 0 1 1 0 0 1 0 +1 179 179 1 0 1 1 0 0 1 1 +1 180 180 1 0 1 1 0 1 0 0 +1 181 181 1 0 1 1 0 1 0 1 +1 182 182 1 0 1 1 0 1 1 0 +1 183 183 1 0 1 1 0 1 1 1 +1 184 184 1 0 1 1 1 0 0 0 +1 185 185 1 0 1 1 1 0 0 1 +1 186 186 1 0 1 1 1 0 1 0 +1 187 187 1 0 1 1 1 0 1 1 +1 188 188 1 0 1 1 1 1 0 0 +1 189 189 1 0 1 1 1 1 0 1 +1 190 190 1 0 1 1 1 1 1 0 +1 191 191 1 0 1 1 1 1 1 1 +1 192 192 1 1 0 0 0 0 0 0 +1 193 193 1 1 0 0 0 0 0 1 +1 194 194 1 1 0 0 0 0 1 0 +1 195 195 1 1 0 0 0 0 1 1 +1 196 196 1 1 0 0 0 1 0 0 +1 197 197 1 1 0 0 0 1 0 1 +1 198 198 1 1 0 0 0 1 1 0 +1 199 199 1 1 0 0 0 1 1 1 +1 200 200 1 1 0 0 1 0 0 0 +1 201 201 1 1 0 0 1 0 0 1 +1 202 202 1 1 0 0 1 0 1 0 +1 203 203 1 1 0 0 1 0 1 1 +1 204 204 1 1 0 0 1 1 0 0 +1 205 205 1 1 0 0 1 1 0 1 +1 206 206 1 1 0 0 1 1 1 0 +1 207 207 1 1 0 0 1 1 1 1 +1 208 208 1 1 0 1 0 0 0 0 +1 209 209 1 1 0 1 0 0 0 1 +1 210 210 1 1 0 1 0 0 1 0 +1 211 211 1 1 0 1 0 0 1 1 +1 212 212 1 1 0 1 0 1 0 0 +1 213 213 1 1 0 1 0 1 0 1 +1 214 214 1 1 0 1 0 1 1 0 +1 215 215 1 1 0 1 0 1 1 1 +1 216 216 1 1 0 1 1 0 0 0 +1 217 217 1 1 0 1 1 0 0 1 +1 218 218 1 1 0 1 1 0 1 0 +1 219 219 1 1 0 1 1 0 1 1 +1 220 220 1 1 0 1 1 1 0 0 +1 221 221 1 1 0 1 1 1 0 1 +1 222 222 1 1 0 1 1 1 1 0 +1 223 223 1 1 0 1 1 1 1 1 +1 224 224 1 1 1 0 0 0 0 0 +1 225 225 1 1 1 0 0 0 0 1 +1 226 226 1 1 1 0 0 0 1 0 +1 227 227 1 1 1 0 0 0 1 1 +1 228 228 1 1 1 0 0 1 0 0 +1 229 229 1 1 1 0 0 1 0 1 +1 230 230 1 1 1 0 0 1 1 0 +1 231 231 1 1 1 0 0 1 1 1 +1 232 232 1 1 1 0 1 0 0 0 +1 233 233 1 1 1 0 1 0 0 1 +1 234 234 1 1 1 0 1 0 1 0 +1 235 235 1 1 1 0 1 0 1 1 +1 236 236 1 1 1 0 1 1 0 0 +1 237 237 1 1 1 0 1 1 0 1 +1 238 238 1 1 1 0 1 1 1 0 +1 239 239 1 1 1 0 1 1 1 1 +1 240 240 1 1 1 1 0 0 0 0 +1 241 241 1 1 1 1 0 0 0 1 +1 242 242 1 1 1 1 0 0 1 0 +1 243 243 1 1 1 1 0 0 1 1 +1 244 244 1 1 1 1 0 1 0 0 +1 245 245 1 1 1 1 0 1 0 1 +1 246 246 1 1 1 1 0 1 1 0 +1 247 247 1 1 1 1 0 1 1 1 +1 248 248 1 1 1 1 1 0 0 0 +1 249 249 1 1 1 1 1 0 0 1 +1 250 250 1 1 1 1 1 0 1 0 +1 251 251 1 1 1 1 1 0 1 1 +1 252 252 1 1 1 1 1 1 0 0 +1 253 253 1 1 1 1 1 1 0 1 +1 254 254 1 1 1 1 1 1 1 0 +1 255 255 1 1 1 1 1 1 1 1 +1 0 0 0 0 0 0 0 0 0 0 +1 1 1 0 0 0 0 0 0 0 1 +1 2 2 0 0 0 0 0 0 1 0 +1 3 3 0 0 0 0 0 0 1 1 +1 4 4 0 0 0 0 0 1 0 0 +1 5 5 0 0 0 0 0 1 0 1 +1 6 6 0 0 0 0 0 1 1 0 +1 7 7 0 0 0 0 0 1 1 1 +1 8 8 0 0 0 0 1 0 0 0 +1 9 9 0 0 0 0 1 0 0 1 +1 10 10 0 0 0 0 1 0 1 0 +1 11 11 0 0 0 0 1 0 1 1 +1 12 12 0 0 0 0 1 1 0 0 +1 13 13 0 0 0 0 1 1 0 1 +1 14 14 0 0 0 0 1 1 1 0 +1 15 15 0 0 0 0 1 1 1 1 +1 16 16 0 0 0 1 0 0 0 0 +1 17 17 0 0 0 1 0 0 0 1 +1 18 18 0 0 0 1 0 0 1 0 +1 19 19 0 0 0 1 0 0 1 1 +1 20 20 0 0 0 1 0 1 0 0 +1 21 21 0 0 0 1 0 1 0 1 +1 22 22 0 0 0 1 0 1 1 0 +1 23 23 0 0 0 1 0 1 1 1 +1 24 24 0 0 0 1 1 0 0 0 +1 25 25 0 0 0 1 1 0 0 1 +1 26 26 0 0 0 1 1 0 1 0 +1 27 27 0 0 0 1 1 0 1 1 +1 28 28 0 0 0 1 1 1 0 0 +1 29 29 0 0 0 1 1 1 0 1 +1 30 30 0 0 0 1 1 1 1 0 +1 31 31 0 0 0 1 1 1 1 1 +1 32 32 0 0 1 0 0 0 0 0 +1 33 33 0 0 1 0 0 0 0 1 +1 34 34 0 0 1 0 0 0 1 0 +1 35 35 0 0 1 0 0 0 1 1 +1 36 36 0 0 1 0 0 1 0 0 +1 37 37 0 0 1 0 0 1 0 1 +1 38 38 0 0 1 0 0 1 1 0 +1 39 39 0 0 1 0 0 1 1 1 +1 40 40 0 0 1 0 1 0 0 0 +1 41 41 0 0 1 0 1 0 0 1 +1 42 42 0 0 1 0 1 0 1 0 +1 43 43 0 0 1 0 1 0 1 1 +1 44 44 0 0 1 0 1 1 0 0 +1 45 45 0 0 1 0 1 1 0 1 +1 46 46 0 0 1 0 1 1 1 0 +1 47 47 0 0 1 0 1 1 1 1 +1 48 48 0 0 1 1 0 0 0 0 +1 49 49 0 0 1 1 0 0 0 1 +1 50 50 0 0 1 1 0 0 1 0 +1 51 51 0 0 1 1 0 0 1 1 +1 52 52 0 0 1 1 0 1 0 0 +1 53 53 0 0 1 1 0 1 0 1 +1 54 54 0 0 1 1 0 1 1 0 +1 55 55 0 0 1 1 0 1 1 1 +1 56 56 0 0 1 1 1 0 0 0 +1 57 57 0 0 1 1 1 0 0 1 +1 58 58 0 0 1 1 1 0 1 0 +1 59 59 0 0 1 1 1 0 1 1 +1 60 60 0 0 1 1 1 1 0 0 +1 61 61 0 0 1 1 1 1 0 1 +1 62 62 0 0 1 1 1 1 1 0 +1 63 63 0 0 1 1 1 1 1 1 +1 64 64 0 1 0 0 0 0 0 0 +1 65 65 0 1 0 0 0 0 0 1 +1 66 66 0 1 0 0 0 0 1 0 +1 67 67 0 1 0 0 0 0 1 1 +1 68 68 0 1 0 0 0 1 0 0 +1 69 69 0 1 0 0 0 1 0 1 +1 70 70 0 1 0 0 0 1 1 0 +1 71 71 0 1 0 0 0 1 1 1 +1 72 72 0 1 0 0 1 0 0 0 +1 73 73 0 1 0 0 1 0 0 1 +1 74 74 0 1 0 0 1 0 1 0 +1 75 75 0 1 0 0 1 0 1 1 +1 76 76 0 1 0 0 1 1 0 0 +1 77 77 0 1 0 0 1 1 0 1 +1 78 78 0 1 0 0 1 1 1 0 +1 79 79 0 1 0 0 1 1 1 1 +1 80 80 0 1 0 1 0 0 0 0 +1 81 81 0 1 0 1 0 0 0 1 +1 82 82 0 1 0 1 0 0 1 0 +1 83 83 0 1 0 1 0 0 1 1 +1 84 84 0 1 0 1 0 1 0 0 +1 85 85 0 1 0 1 0 1 0 1 +1 86 86 0 1 0 1 0 1 1 0 +1 87 87 0 1 0 1 0 1 1 1 +1 88 88 0 1 0 1 1 0 0 0 +1 89 89 0 1 0 1 1 0 0 1 +1 90 90 0 1 0 1 1 0 1 0 +1 91 91 0 1 0 1 1 0 1 1 +1 92 92 0 1 0 1 1 1 0 0 +1 93 93 0 1 0 1 1 1 0 1 +1 94 94 0 1 0 1 1 1 1 0 +1 95 95 0 1 0 1 1 1 1 1 +1 96 96 0 1 1 0 0 0 0 0 +1 97 97 0 1 1 0 0 0 0 1 +1 98 98 0 1 1 0 0 0 1 0 +1 99 99 0 1 1 0 0 0 1 1 +1 100 100 0 1 1 0 0 1 0 0 +1 101 101 0 1 1 0 0 1 0 1 +1 102 102 0 1 1 0 0 1 1 0 +1 103 103 0 1 1 0 0 1 1 1 +1 104 104 0 1 1 0 1 0 0 0 +1 105 105 0 1 1 0 1 0 0 1 +1 106 106 0 1 1 0 1 0 1 0 +1 107 107 0 1 1 0 1 0 1 1 +1 108 108 0 1 1 0 1 1 0 0 +1 109 109 0 1 1 0 1 1 0 1 +1 110 110 0 1 1 0 1 1 1 0 +1 111 111 0 1 1 0 1 1 1 1 +1 112 112 0 1 1 1 0 0 0 0 +1 113 113 0 1 1 1 0 0 0 1 +1 114 114 0 1 1 1 0 0 1 0 +1 115 115 0 1 1 1 0 0 1 1 +1 116 116 0 1 1 1 0 1 0 0 +1 117 117 0 1 1 1 0 1 0 1 +1 118 118 0 1 1 1 0 1 1 0 +1 119 119 0 1 1 1 0 1 1 1 +1 120 120 0 1 1 1 1 0 0 0 +1 121 121 0 1 1 1 1 0 0 1 +1 122 122 0 1 1 1 1 0 1 0 +1 123 123 0 1 1 1 1 0 1 1 +1 124 124 0 1 1 1 1 1 0 0 +1 125 125 0 1 1 1 1 1 0 1 +1 126 126 0 1 1 1 1 1 1 0 +1 127 127 0 1 1 1 1 1 1 1 +1 128 128 1 0 0 0 0 0 0 0 +1 129 129 1 0 0 0 0 0 0 1 +1 130 130 1 0 0 0 0 0 1 0 +1 131 131 1 0 0 0 0 0 1 1 +1 132 132 1 0 0 0 0 1 0 0 +1 133 133 1 0 0 0 0 1 0 1 +1 134 134 1 0 0 0 0 1 1 0 +1 135 135 1 0 0 0 0 1 1 1 +1 136 136 1 0 0 0 1 0 0 0 +1 137 137 1 0 0 0 1 0 0 1 +1 138 138 1 0 0 0 1 0 1 0 +1 139 139 1 0 0 0 1 0 1 1 +1 140 140 1 0 0 0 1 1 0 0 +1 141 141 1 0 0 0 1 1 0 1 +1 142 142 1 0 0 0 1 1 1 0 +1 143 143 1 0 0 0 1 1 1 1 +1 144 144 1 0 0 1 0 0 0 0 +1 145 145 1 0 0 1 0 0 0 1 +1 146 146 1 0 0 1 0 0 1 0 +1 147 147 1 0 0 1 0 0 1 1 +1 148 148 1 0 0 1 0 1 0 0 +1 149 149 1 0 0 1 0 1 0 1 +1 150 150 1 0 0 1 0 1 1 0 +1 151 151 1 0 0 1 0 1 1 1 +1 152 152 1 0 0 1 1 0 0 0 +1 153 153 1 0 0 1 1 0 0 1 +1 154 154 1 0 0 1 1 0 1 0 +1 155 155 1 0 0 1 1 0 1 1 +1 156 156 1 0 0 1 1 1 0 0 +1 157 157 1 0 0 1 1 1 0 1 +1 158 158 1 0 0 1 1 1 1 0 +1 159 159 1 0 0 1 1 1 1 1 +1 160 160 1 0 1 0 0 0 0 0 +1 161 161 1 0 1 0 0 0 0 1 +1 162 162 1 0 1 0 0 0 1 0 +1 163 163 1 0 1 0 0 0 1 1 +1 164 164 1 0 1 0 0 1 0 0 +1 165 165 1 0 1 0 0 1 0 1 +1 166 166 1 0 1 0 0 1 1 0 +1 167 167 1 0 1 0 0 1 1 1 +1 168 168 1 0 1 0 1 0 0 0 +1 169 169 1 0 1 0 1 0 0 1 +1 170 170 1 0 1 0 1 0 1 0 +1 171 171 1 0 1 0 1 0 1 1 +1 172 172 1 0 1 0 1 1 0 0 +1 173 173 1 0 1 0 1 1 0 1 +1 174 174 1 0 1 0 1 1 1 0 +1 175 175 1 0 1 0 1 1 1 1 +1 176 176 1 0 1 1 0 0 0 0 +1 177 177 1 0 1 1 0 0 0 1 +1 178 178 1 0 1 1 0 0 1 0 +1 179 179 1 0 1 1 0 0 1 1 +1 180 180 1 0 1 1 0 1 0 0 +1 181 181 1 0 1 1 0 1 0 1 +1 182 182 1 0 1 1 0 1 1 0 +1 183 183 1 0 1 1 0 1 1 1 +1 184 184 1 0 1 1 1 0 0 0 +1 185 185 1 0 1 1 1 0 0 1 +1 186 186 1 0 1 1 1 0 1 0 +1 187 187 1 0 1 1 1 0 1 1 +1 188 188 1 0 1 1 1 1 0 0 +1 189 189 1 0 1 1 1 1 0 1 +1 190 190 1 0 1 1 1 1 1 0 +1 191 191 1 0 1 1 1 1 1 1 +1 192 192 1 1 0 0 0 0 0 0 +1 193 193 1 1 0 0 0 0 0 1 +1 194 194 1 1 0 0 0 0 1 0 +1 195 195 1 1 0 0 0 0 1 1 +1 196 196 1 1 0 0 0 1 0 0 +1 197 197 1 1 0 0 0 1 0 1 +1 198 198 1 1 0 0 0 1 1 0 +1 199 199 1 1 0 0 0 1 1 1 +1 200 200 1 1 0 0 1 0 0 0 +1 201 201 1 1 0 0 1 0 0 1 +1 202 202 1 1 0 0 1 0 1 0 +1 203 203 1 1 0 0 1 0 1 1 +1 204 204 1 1 0 0 1 1 0 0 +1 205 205 1 1 0 0 1 1 0 1 +1 206 206 1 1 0 0 1 1 1 0 +1 207 207 1 1 0 0 1 1 1 1 +1 208 208 1 1 0 1 0 0 0 0 +1 209 209 1 1 0 1 0 0 0 1 +1 210 210 1 1 0 1 0 0 1 0 +1 211 211 1 1 0 1 0 0 1 1 +1 212 212 1 1 0 1 0 1 0 0 +1 213 213 1 1 0 1 0 1 0 1 +1 214 214 1 1 0 1 0 1 1 0 +1 215 215 1 1 0 1 0 1 1 1 +1 216 216 1 1 0 1 1 0 0 0 +1 217 217 1 1 0 1 1 0 0 1 +1 218 218 1 1 0 1 1 0 1 0 +1 219 219 1 1 0 1 1 0 1 1 +1 220 220 1 1 0 1 1 1 0 0 +1 221 221 1 1 0 1 1 1 0 1 +1 222 222 1 1 0 1 1 1 1 0 +1 223 223 1 1 0 1 1 1 1 1 +1 224 224 1 1 1 0 0 0 0 0 +1 225 225 1 1 1 0 0 0 0 1 +1 226 226 1 1 1 0 0 0 1 0 +1 227 227 1 1 1 0 0 0 1 1 +1 228 228 1 1 1 0 0 1 0 0 +1 229 229 1 1 1 0 0 1 0 1 +1 230 230 1 1 1 0 0 1 1 0 +1 231 231 1 1 1 0 0 1 1 1 +1 232 232 1 1 1 0 1 0 0 0 +1 233 233 1 1 1 0 1 0 0 1 +1 234 234 1 1 1 0 1 0 1 0 +1 235 235 1 1 1 0 1 0 1 1 +1 236 236 1 1 1 0 1 1 0 0 +1 237 237 1 1 1 0 1 1 0 1 +1 238 238 1 1 1 0 1 1 1 0 +1 239 239 1 1 1 0 1 1 1 1 +1 240 240 1 1 1 1 0 0 0 0 +1 241 241 1 1 1 1 0 0 0 1 +1 242 242 1 1 1 1 0 0 1 0 +1 243 243 1 1 1 1 0 0 1 1 +1 244 244 1 1 1 1 0 1 0 0 +1 245 245 1 1 1 1 0 1 0 1 +1 246 246 1 1 1 1 0 1 1 0 +1 247 247 1 1 1 1 0 1 1 1 +1 248 248 1 1 1 1 1 0 0 0 +1 249 249 1 1 1 1 1 0 0 1 +1 250 250 1 1 1 1 1 0 1 0 +1 251 251 1 1 1 1 1 0 1 1 +1 252 252 1 1 1 1 1 1 0 0 +1 253 253 1 1 1 1 1 1 0 1 +1 254 254 1 1 1 1 1 1 1 0 +1 255 255 1 1 1 1 1 1 1 1 diff --git a/dbms/tests/queries/0_stateless/00216_bit_test_function_family.sql b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.sql new file mode 100644 index 00000000000..a8bbc379087 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00216_bit_test_function_family.sql @@ -0,0 +1,115 @@ +select + bitTest(0, 0) = 0, + bitTest(1, 0) = 1, + bitTest(1, 1) = 0, + bitTest(0xff, 7) = 1; + +select + bitTestAll(0, 0) = 0, + bitTestAll(1, 0) = 1, + bitTestAll(1, 1) = 0, + bitTestAll(0xff, 0) = 1, + bitTestAll(0xff, 1) = 1, + bitTestAll(0xff, 2) = 1, + bitTestAll(0xff, 3) = 1, + bitTestAll(0xff, 4) = 1, + bitTestAll(0xff, 5) = 1, + bitTestAll(0xff, 6) = 1, + bitTestAll(0xff, 7) = 1, + bitTestAll(0xff, 0, 1) = 1, + bitTestAll(0xff, 2, 3) = 1, + bitTestAll(0xff, 4, 5) = 1, + bitTestAll(0xff, 6, 7) = 1, + bitTestAll(0xff, 0, 1, 2, 3) = 1, + bitTestAll(0xff, 4, 5, 6, 7) = 1, + bitTestAll(0xff, 0, 1, 2, 3, 4, 5, 6, 7) = 1, + bitTestAll(0x81, 0) = 1, + bitTestAll(0x81, 1) = 0, + bitTestAll(0x81, 2) = 0, + bitTestAll(0x81, 3) = 0, + bitTestAll(0x81, 4) = 0, + bitTestAll(0x81, 5) = 0, + bitTestAll(0x81, 6) = 0, + bitTestAll(0x81, 7) = 1, + bitTestAll(0x81, 0, 1) = 0, + bitTestAll(0x81, 2, 3) = 0, + bitTestAll(0x81, 4, 5) = 0, + bitTestAll(0x81, 6, 7) = 0, + bitTestAll(0x81, 0, 1, 2, 3) = 0, + bitTestAll(0x81, 4, 5, 6, 7) = 0, + bitTestAll(0x81, 0, 1, 2, 3, 4, 5, 6, 7) = 0, + bitTestAll(0x81, 0, 7) = 1; + +select + bitTestAny(0, 0) = 0, + bitTestAny(1, 0) = 1, + bitTestAny(1, 1) = 0, + bitTestAny(0xff, 0) = 1, + bitTestAny(0xff, 1) = 1, + bitTestAny(0xff, 2) = 1, + bitTestAny(0xff, 3) = 1, + bitTestAny(0xff, 4) = 1, + bitTestAny(0xff, 5) = 1, + bitTestAny(0xff, 6) = 1, + bitTestAny(0xff, 7) = 1, + bitTestAny(0xff, 0, 1) = 1, + bitTestAny(0xff, 2, 3) = 1, + bitTestAny(0xff, 4, 5) = 1, + bitTestAny(0xff, 6, 7) = 1, + bitTestAny(0xff, 0, 1, 2, 3) = 1, + bitTestAny(0xff, 4, 5, 6, 7) = 1, + bitTestAny(0xff, 0, 1, 2, 3, 4, 5, 6, 7) = 1, + bitTestAny(0x81, 0) = 1, + bitTestAny(0x81, 1) = 0, + bitTestAny(0x81, 2) = 0, + bitTestAny(0x81, 3) = 0, + bitTestAny(0x81, 4) = 0, + bitTestAny(0x81, 5) = 0, + bitTestAny(0x81, 6) = 0, + bitTestAny(0x81, 7) = 1, + bitTestAny(0x81, 0, 1) = 1, + bitTestAny(0x81, 2, 3) = 0, + bitTestAny(0x81, 4, 5) = 0, + bitTestAny(0x81, 6, 7) = 1, + bitTestAny(0x81, 0, 1, 2, 3) = 1, + bitTestAny(0x81, 4, 5, 6, 7) = 1, + bitTestAny(0x81, 0, 1, 2, 3, 4, 5, 6, 7) = 1; + +select n = n_, + number as n, + bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(b7, 1), b6), 1), b5), 1), b4), 1), b3), 1), b2), 1), b1), 1), b0) as n_, + bitTest(n, 7) as b7, + bitTest(n, 6) as b6, + bitTest(n, 5) as b5, + bitTest(n, 4) as b4, + bitTest(n, 3) as b3, + bitTest(n, 2) as b2, + bitTest(n, 1) as b1, + bitTest(n, 0) as b0 +from system.numbers limit 256; + +select n = n_, + number as n, + bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(b7, 1), b6), 1), b5), 1), b4), 1), b3), 1), b2), 1), b1), 1), b0) as n_, + bitTestAll(n, 7) as b7, + bitTestAll(n, 6) as b6, + bitTestAll(n, 5) as b5, + bitTestAll(n, 4) as b4, + bitTestAll(n, 3) as b3, + bitTestAll(n, 2) as b2, + bitTestAll(n, 1) as b1, + bitTestAll(n, 0) as b0 +from system.numbers limit 256; + +select n = n_, + number as n, + bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(bitOr(bitShiftLeft(b7, 1), b6), 1), b5), 1), b4), 1), b3), 1), b2), 1), b1), 1), b0) as n_, + bitTestAny(n, 7) as b7, + bitTestAny(n, 6) as b6, + bitTestAny(n, 5) as b5, + bitTestAny(n, 4) as b4, + bitTestAny(n, 3) as b3, + bitTestAny(n, 2) as b2, + bitTestAny(n, 1) as b1, + bitTestAny(n, 0) as b0 +from system.numbers limit 256; From 914e74f796d63fe5b9d3f602922f462995db91e7 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 20 Aug 2015 19:47:02 +0300 Subject: [PATCH 11/50] dbms: fix quantilesTiming for levels specified in non-increasing order [#METR-14503] --- .../AggregateFunctionQuantileTiming.h | 45 ++++++++++--------- 1 file changed, 25 insertions(+), 20 deletions(-) diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h index fcc47a9b8dc..86a75c339e0 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionQuantileTiming.h @@ -15,6 +15,7 @@ #include #include +#include namespace DB @@ -234,9 +235,16 @@ namespace detail template void getMany(const double * levels, size_t size, ResultType * result) const { - const double * levels_end = levels + size; - const double * level = levels; - UInt64 pos = count * *level; + std::size_t indices[size]; + std::copy(ext::range_iterator{}, ext::make_range_iterator(size), indices); + std::sort(indices, indices + size, [levels] (auto i1, auto i2) { + return levels[i1] < levels[i2]; + }); + + const auto indices_end = indices + size; + auto index = indices; + + UInt64 pos = count * levels[*index]; UInt64 accumulated = 0; @@ -251,15 +259,14 @@ namespace detail if (i < SMALL_THRESHOLD) { - *result = i; + result[*index] = i; - ++level; - ++result; + ++index; - if (level == levels_end) + if (index == indices_end) return; - pos = count * *level; + pos = count * levels[*index]; } } @@ -274,24 +281,22 @@ namespace detail if (i < BIG_SIZE) { - *result = indexInBigToValue(i); + result[*index] = indexInBigToValue(i); - ++level; - ++result; + ++index; - if (level == levels_end) + if (index == indices_end) return; - pos = count * *level; + pos = count * levels[*index]; } } - while (level < levels_end) + while (index < indices_end) { - *result = BIG_THRESHOLD; + result[*index] = BIG_THRESHOLD; - ++level; - ++result; + ++index; } } @@ -466,7 +471,7 @@ public: large = new detail::QuantileTimingLarge; } - + large->merge(detail::QuantileTimingLarge(buf)); } else @@ -712,11 +717,11 @@ public: size_t size = levels.size(); offsets_to.push_back((offsets_to.size() == 0 ? 0 : offsets_to.back()) + size); - + typename ColumnFloat32::Container_t & data_to = static_cast(arr_to.getData()).getData(); size_t old_size = data_to.size(); data_to.resize(data_to.size() + size); - + this->data(place).getManyFloat(&levels[0], size, &data_to[old_size]); } }; From d43a8cce1f3ba3fdd9eccd7033df82053b8e1f96 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Fri, 21 Aug 2015 12:16:52 +0300 Subject: [PATCH 12/50] =?UTF-8?q?fix=20linking=20of=20=E2=96=88=E2=96=88?= =?UTF-8?q?=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88?= =?UTF-8?q?=E2=96=88=E2=96=88?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbms/include/DB/Functions/FunctionsCoding.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index b4aaddeca1f..1d297bea127 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -1569,7 +1569,7 @@ template struct FunctionBitTestMany : public IFunction { public: - static constexpr auto && name = Impl::name; + static constexpr auto name = Impl::name; static IFunction * create(const Context &) { return new FunctionBitTestMany; } String getName() const override { return name; } From acb10923ded17bc3df850a6e28e1a733c0cb59af Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Fri, 21 Aug 2015 18:57:26 +0300 Subject: [PATCH 13/50] add aggregate function sequenceCount [#METR-17427] --- .../AggregateFunctionSequenceMatch.h | 73 +++++++++++-------- .../AggregateFunctionFactory.cpp | 8 ++ 2 files changed, 52 insertions(+), 29 deletions(-) diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h index 255028c2d05..aef08825536 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -124,7 +124,7 @@ struct AggregateFunctionSequenceMatchData final /// Max number of iterations to match the pattern against a sequence, exception thrown when exceeded constexpr auto sequence_match_max_iterations = 1000000; -class AggregateFunctionSequenceMatch final : public IAggregateFunctionHelper +class AggregateFunctionSequenceMatch : public IAggregateFunctionHelper { public: static bool sufficientArgs(const std::size_t arg_count) { return arg_count >= 3; } @@ -218,7 +218,14 @@ public: void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override { const_cast(data(place)).sort(); - static_cast(to).getData().push_back(match(place)); + + const auto & data_ref = data(place); + + const auto events_begin = std::begin(data_ref.eventsList); + const auto events_end = std::end(data_ref.eventsList); + auto events_it = events_begin; + + static_cast(to).getData().push_back(match(events_it, events_end)); } private: @@ -233,21 +240,6 @@ private: TimeGreater }; - static std::string to_string(const PatternActionType type) - { - static const std::map map{ - { PatternActionType::SpecificEvent, "SpecificEvent" }, - { PatternActionType::AnyEvent, "AnyEvent" }, - { PatternActionType::KleeneStar, "KleeneStar" }, - { PatternActionType::TimeLessOrEqual, "TimeLessOrEqual" }, - { PatternActionType::TimeLess, "TimeLess", }, - { PatternActionType::TimeGreaterOrEqual, "TimeGreaterOrEqual" }, - { PatternActionType::TimeGreater, "TimeGreater" } - }; - - return map.find(type)->second; - } - struct PatternAction final { PatternActionType type; @@ -353,18 +345,15 @@ private: this->actions = std::move(actions); } - bool match(const ConstAggregateDataPtr & place) const +protected: + template + bool match(T1 & events_it, const T2 events_end) const { const auto action_begin = std::begin(actions); const auto action_end = std::end(actions); auto action_it = action_begin; - const auto & data_ref = data(place); - const auto events_begin = std::begin(data_ref.eventsList); - const auto events_end = std::end(data_ref.eventsList); - auto events_it = events_begin; - - auto base_it = events_begin; + auto base_it = events_it; /// an iterator to action plus an iterator to row in events list plus timestamp at the start of sequence using backtrack_info = std::tuple; @@ -392,11 +381,6 @@ private: std::size_t i = 0; while (action_it != action_end && events_it != events_end) { -// std::cout << "start_timestamp " << base_it->first << "; "; -// std::cout << "elapsed " << (events_it->first - base_it->first) << "; "; -// std::cout << "action " << (action_it - action_begin) << " { " << to_string(action_it->type) << ' ' << action_it->extra << " }; "; -// std::cout << "symbol " << (events_it - events_begin) << " { " << events_it->first << ' ' << events_it->second.to_ulong() << " }" << std::endl; - if (action_it->type == PatternActionType::SpecificEvent) { if (events_it->second.test(action_it->extra)) @@ -492,9 +476,40 @@ private: return action_it == action_end; } +private: std::string pattern; std::size_t arg_count; PatternActions actions; }; +class AggregateFunctionSequenceCount final : public AggregateFunctionSequenceMatch +{ +public: + String getName() const override { return "sequenceCount"; } + + DataTypePtr getReturnType() const override { return new DataTypeUInt64; } + + void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override + { + const_cast(data(place)).sort(); + static_cast(to).getData().push_back(count(place)); + } + +private: + UInt64 count(const ConstAggregateDataPtr & place) const + { + const auto & data_ref = data(place); + + const auto events_begin = std::begin(data_ref.eventsList); + const auto events_end = std::end(data_ref.eventsList); + auto events_it = events_begin; + + std::size_t count = 0; + while (events_it != events_end && match(events_it, events_end)) + ++count; + + return count; + } +}; + } diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 245530b86d6..c5f503da3f0 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -562,6 +562,13 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da return new AggregateFunctionSequenceMatch; } + else if (name == "sequenceCount") + { + if (!AggregateFunctionSequenceCount::sufficientArgs(argument_types.size())) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return new AggregateFunctionSequenceCount; + } else if (name == "varSamp") { if (argument_types.size() != 1) @@ -743,6 +750,7 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun "quantileDeterministic", "quantilesDeterministic", "sequenceMatch", + "sequenceCount", "varSamp", "varPop", "stddevSamp", From fbfaa83b1a957fa11f9ec6b71681e503e3944e36 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Tue, 25 Aug 2015 12:31:55 +0300 Subject: [PATCH 14/50] Merge --- dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h | 2 +- .../Storages/MergeTree/MergeTreeThreadBlockInputStream.h | 8 +++----- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index d1766c27e60..8750f55b283 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -121,7 +121,7 @@ public: } return std::make_unique( - part.data_part, thread_task.ranges, part.part_index_in_query, column_names, + part.data_part, ranges_to_get_from_part, part.part_index_in_query, column_names, per_part_column_name_set[part_idx], per_part_columns[part_idx], per_part_pre_columns[part_idx], per_part_remove_prewhere_column[part_idx], per_part_should_reorder[part_idx]); } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 22ac5e9822b..77f39346e45 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -69,12 +69,7 @@ protected: injectVirtualColumns(res); if (task->mark_ranges.empty()) - { - /** Закрываем файлы (ещё до уничтожения объекта). - * Чтобы при создании многих источников, но одновременном чтении только из нескольких, - * буферы не висели в памяти. */ task = {}; - } } return res; @@ -87,6 +82,9 @@ private: if (!task) { + /** Закрываем файлы (ещё до уничтожения объекта). + * Чтобы при создании многих источников, но одновременном чтении только из нескольких, + * буферы не висели в памяти. */ reader = {}; pre_reader = {}; return false; From 1288087809aa40c0007a25788001d7a82b0e86b8 Mon Sep 17 00:00:00 2001 From: Vsevolod Orlov Date: Wed, 26 Aug 2015 12:19:32 +0300 Subject: [PATCH 15/50] Merge From f93141355b41327f9271524382bf86c6c8002e32 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Mon, 31 Aug 2015 16:52:17 +0300 Subject: [PATCH 16/50] Merge --- .../AggregateFunctionUniq.h | 209 +++++----- .../AggregateFunctions/UniqCombinedBiasData.h | 39 ++ .../DB/Common/CombinedCardinalityEstimator.h | 19 +- .../HyperLogLogWithSmallSetOptimization.h | 2 +- dbms/scripts/gen-bias-data.py | 252 +++++++++++ dbms/scripts/linear-counting-threshold.py | 143 +++++++ .../AggregateFunctionFactory.cpp | 69 ++++ .../UniqCombinedBiasData.cpp | 390 ++++++++++++++++++ .../00212_aggregate_function_uniq.reference | 114 ++--- 9 files changed, 1066 insertions(+), 171 deletions(-) create mode 100644 dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h create mode 100755 dbms/scripts/gen-bias-data.py create mode 100755 dbms/scripts/linear-counting-threshold.py create mode 100644 dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h index 3975f238818..6a22bbb1a2b 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h @@ -20,6 +20,7 @@ #include #include +#include namespace DB @@ -29,7 +30,7 @@ namespace DB struct AggregateFunctionUniqUniquesHashSetData { - typedef UniquesHashSet> Set; + typedef UniquesHashSet > Set; Set set; static String getName() { return "uniq"; } @@ -94,83 +95,79 @@ struct AggregateFunctionUniqExactData static String getName() { return "uniqExact"; } }; +template +struct BaseUniqCombinedData +{ + using Key = UInt64; + using Set = CombinedCardinalityEstimator< + Key, + HashSet, HashTableGrower<> >, + 16, + 14, + 17, + DefaultHash, + UInt64, + HyperLogLogBiasEstimator, + mode + >; + + Set set; +}; + +template +struct BaseUniqCombinedData +{ + using Key = UInt64; + using Set = CombinedCardinalityEstimator< + Key, + HashSet >, + 16, + 14, + 17, + TrivialHash, + UInt64, + HyperLogLogBiasEstimator, + mode + >; + + Set set; +}; + +/// Агрегатные функции uniqCombinedRaw, uniqCombinedLinearCounting, и uniqCombinedBiasCorrected +/// предназначены для разработки новых версий функции uniqCombined. +/// Пользователи должны использовать только uniqCombined. + +template +struct AggregateFunctionUniqCombinedRawData + : public BaseUniqCombinedData +{ + static String getName() { return "uniqCombinedRaw"; } +}; + +template +struct AggregateFunctionUniqCombinedLinearCountingData + : public BaseUniqCombinedData +{ + static String getName() { return "uniqCombinedLinearCounting"; } +}; + +template +struct AggregateFunctionUniqCombinedBiasCorrectedData + : public BaseUniqCombinedData +{ + static String getName() { return "uniqCombinedBiasCorrected"; } +}; template struct AggregateFunctionUniqCombinedData + : public BaseUniqCombinedData { - using Key = UInt32; - using Set = CombinedCardinalityEstimator >, 16, 14, 17, TrivialHash>; - Set set; - - static String getName() { return "uniqCombined"; } -}; - -template <> -struct AggregateFunctionUniqCombinedData -{ - using Key = UInt64; - using Set = CombinedCardinalityEstimator >, 16, 14, 17, TrivialHash>; - Set set; - static String getName() { return "uniqCombined"; } }; namespace detail { -/** Хэш-функция для uniqCombined. - */ -template -struct CombinedCardinalityTraits -{ - static UInt32 hash(T key) - { - return key; - } -}; - -template -struct CombinedCardinalityTraits::value>::type> -{ - using U = typename std::make_unsigned::type; - - static UInt32 hash(T key) - { - return intHash32<0>(static_cast(key)); - }; -}; - -template -struct CombinedCardinalityTraits::value>::type> -{ - static UInt32 hash(T key) - { - return intHash32<0>(key); - }; -}; - -template -struct CombinedCardinalityTraits::value>::type> -{ - static UInt32 hash(T key) - { - UInt64 res = 0; - memcpy(reinterpret_cast(&res), reinterpret_cast(&key), sizeof(key)); - return intHash32<0>(res); - } -}; - -template -struct CombinedCardinalityTraits::value>::type> -{ - static UInt32 hash(T key) - { - UInt32 res = 0; - memcpy(reinterpret_cast(&res), reinterpret_cast(&key), sizeof(key)); - return res; - } -}; - /** Хэш-функция для uniq. */ template struct AggregateFunctionUniqTraits @@ -199,41 +196,51 @@ template <> struct AggregateFunctionUniqTraits }; /** Структура для делегации работы по добавлению одного элемента в агрегатные функции uniq. - * Используется для частичной специализации для добавления строк. - */ -template -struct OneAdder -{ - static void addOne(Data & data, const IColumn & column, size_t row_num) - { - data.set.insert(AggregateFunctionUniqTraits::hash(static_cast &>(column).getData()[row_num])); - } -}; + * Используется для частичной специализации для добавления строк. + */ +template +struct OneAdder; -template -struct OneAdder +template +struct OneAdder::value || + std::is_same >::value || + std::is_same >::value || + std::is_same >::value || + std::is_same >::value || + std::is_same >::value>::type> { - static void addOne(Data & data, const IColumn & column, size_t row_num) + template + static void addOne(Data & data, const IColumn & column, size_t row_num, + typename std::enable_if::value>::type * = nullptr) + { + const auto & value = static_cast &>(column).getData()[row_num]; + data.set.insert(AggregateFunctionUniqTraits::hash(value)); + } + + template + static void addOne(Data & data, const IColumn & column, size_t row_num, + typename std::enable_if::value>::type * = nullptr) { - /// Имейте ввиду, что вычисление приближённое. StringRef value = column.getDataAt(row_num); data.set.insert(CityHash64(value.data, value.size)); } }; -template -struct OneAdder > +template +struct OneAdder>::value>::type> { - static void addOne(AggregateFunctionUniqExactData & data, const IColumn & column, size_t row_num) + template + static void addOne(Data & data, const IColumn & column, size_t row_num, + typename std::enable_if::value>::type * = nullptr) { - data.set.insert(static_cast &>(column).getData()[row_num]); + data.set.insert(static_cast &>(column).getData()[row_num]); } -}; -template<> -struct OneAdder > -{ - static void addOne(AggregateFunctionUniqExactData & data, const IColumn & column, size_t row_num) + template + static void addOne(Data & data, const IColumn & column, size_t row_num, + typename std::enable_if::value>::type * = nullptr) { StringRef value = column.getDataAt(row_num); @@ -246,26 +253,6 @@ struct OneAdder > } }; -template -struct OneAdder > -{ - static void addOne(AggregateFunctionUniqCombinedData & data, const IColumn & column, size_t row_num) - { - const auto & value = static_cast &>(column).getData()[row_num]; - data.set.insert(CombinedCardinalityTraits::hash(value)); - } -}; - -template<> -struct OneAdder > -{ - static void addOne(AggregateFunctionUniqCombinedData & data, const IColumn & column, size_t row_num) - { - StringRef value = column.getDataAt(row_num); - data.set.insert(CityHash64(value.data, value.size)); - } -}; - } diff --git a/dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h b/dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h new file mode 100644 index 00000000000..59f809fb72e --- /dev/null +++ b/dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h @@ -0,0 +1,39 @@ +#pragma once + +#include + +namespace DB +{ + +/** Данные для HyperLogLogBiasEstimator в функции uniqCombined. + * Схема разработки следующая: + * 1. Собрать clickhouse. + * 2. Запустить скрипт src/dbms/scripts/gen-bias-data.py, который возвращает один массив для getRawEstimates() + * и другой массив для getBiases(). + * 3. Обновить массивы raw_estimates и biases. Также обновить размер массивов в InterpolatedData. + * 4. Собрать clickhouse. + * 5. Запустить скрипт src/dbms/scripts/linear-counting-threshold.py, который создаёт 3 файла: + * - raw_graph.txt (1-й столбец: настоящее количество уникальных значений; + * 2-й столбец: относительная погрешность в случае HyperLogLog без применения каких-либо поправок) + * - linear_counting_graph.txt (1-й столбец: настоящее количество уникальных значений; + * 2-й столбец: относительная погрешность в случае HyperLogLog с применением LinearCounting) + * - bias_corrected_graph.txt (1-й столбец: настоящее количество уникальных значений; + * 2-й столбец: относительная погрешность в случае HyperLogLog с применением поправок из алгортима HyperLogLog++) + * 6. Сгенерить график с gnuplot на основе этих данных. + * 7. Определить минимальное количество уникальных значений, при котором лучше исправить погрешность + * с помощью её оценки (т.е. по алгоритму HyperLogLog++), чем применить алгоритм LinearCounting. + * 7. Соответственно обновить константу в функции getThreshold() + * 8. Собрать clickhouse. + */ +struct UniqCombinedBiasData +{ + using InterpolatedData = std::array; + + static double getThreshold(); + /// Оценки количества уникальных значений по алгоритму HyperLogLog без применения каких-либо поправок. + static const InterpolatedData & getRawEstimates(); + /// Соответствующие оценки погрешности. + static const InterpolatedData & getBiases(); +}; + +} diff --git a/dbms/include/DB/Common/CombinedCardinalityEstimator.h b/dbms/include/DB/Common/CombinedCardinalityEstimator.h index c6a2ff2d119..57050bbfb99 100644 --- a/dbms/include/DB/Common/CombinedCardinalityEstimator.h +++ b/dbms/include/DB/Common/CombinedCardinalityEstimator.h @@ -34,17 +34,32 @@ template UInt8 medium_set_power2_max, UInt8 K, typename Hash = IntHash32, + typename HashValueType = UInt32, + typename BiasEstimator = TrivialBiasEstimator, + HyperLogLogMode mode = HyperLogLogMode::FullFeatured, typename DenominatorType = double > class CombinedCardinalityEstimator { public: - using Self = CombinedCardinalityEstimator; + using Self = CombinedCardinalityEstimator + < + Key, + HashContainer, + small_set_size_max, + medium_set_power2_max, + K, + Hash, + HashValueType, + BiasEstimator, + mode, + DenominatorType + >; private: using Small = SmallSet; using Medium = HashContainer; - using Large = HyperLogLogCounter; + using Large = HyperLogLogCounter; public: CombinedCardinalityEstimator() diff --git a/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h b/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h index e95811ce27b..6e8467f0033 100644 --- a/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h +++ b/dbms/include/DB/Common/HyperLogLogWithSmallSetOptimization.h @@ -21,7 +21,7 @@ class HyperLogLogWithSmallSetOptimization { private: using Small = SmallSet; - using Large = HyperLogLogCounter; + using Large = HyperLogLogCounter; Small small; Large * large = nullptr; diff --git a/dbms/scripts/gen-bias-data.py b/dbms/scripts/gen-bias-data.py new file mode 100755 index 00000000000..2d034b0976c --- /dev/null +++ b/dbms/scripts/gen-bias-data.py @@ -0,0 +1,252 @@ +#!/usr/bin/python3.4 +# -*- coding: utf-8 -*- + +import sys +import argparse +import tempfile +import random +import subprocess +import bisect +from copy import deepcopy + +# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/ +class UniqueRandomGenerator: + prime = 4294967291 + + def __init__(self, seed_base, seed_offset): + self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161) + self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905) + + def next(self): + val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635) + self.index = self.index + 1 + return val + + def permutePQR(self, x): + if x >=self.prime: + return x + else: + residue = (x * x) % self.prime + if x <= self.prime/2: + return residue + else: + return self.prime - residue + +def generate_data_source(host, port, http_port, begin, end, count): + chunk_size = round((end - begin) / float(count)) + used_values = 0 + + cur_count = 0 + next_size = 0 + + sup = 32768 + n1 = random.randrange(0, sup) + n2 = random.randrange(0, sup) + urng = UniqueRandomGenerator(n1, n2) + + with tempfile.TemporaryDirectory() as tmp_dir: + filename = tmp_dir + '/table.txt' + file_handle = open(filename, 'w+b') + + while cur_count < count: + next_size += chunk_size + + while used_values < next_size: + h = urng.next() + used_values = used_values + 1 + outstr = str(h) + "\t" + str(cur_count) + "\n"; + file_handle.write(bytes(outstr, 'UTF-8')); + + cur_count = cur_count + 1 + + file_handle.close() + + query = 'DROP TABLE IF EXISTS data_source' + subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) + query = 'CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog' + subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) + + cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE) + subprocess.check_output(("POST", "http://localhost:{0}/?query=INSERT INTO data_source FORMAT TabSeparated".format(http_port)), stdin=cat.stdout) + cat.wait() + +def perform_query(host, port): + query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, " + query += "runningAccumulate(uniqCombinedRawState(UserID)) AS approx " + query += "FROM data_source GROUP BY KeyID" + return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query]) + +def parse_clickhouse_response(output): + parsed = [] + lines = output.decode().split("\n") + for cur_line in lines: + rows = cur_line.split("\t") + if len(rows) == 2: + parsed.append([float(rows[0]), float(rows[1])]) + return parsed + +def accumulate_data(stats, data): + if not stats: + stats = deepcopy(data) + else: + for row1, row2 in zip(stats, data): + row1[1] += row2[1]; + return stats + +def generate_raw_result(stats, count): + expected_tab = [] + bias_tab = [] + for row in stats: + exact = row[0] + expected = row[1] / count + bias = expected - exact + + expected_tab.append(expected) + bias_tab.append(bias) + return [ expected_tab, bias_tab ] + +def generate_sample(raw_estimates, biases, n_samples): + result = [] + + min_card = raw_estimates[0] + max_card = raw_estimates[len(raw_estimates) - 1] + step = (max_card - min_card) / n_samples + + for i in range(0, n_samples + 1): + x = min_card + i * step + j = bisect.bisect_left(raw_estimates, x) + + if j == len(raw_estimates): + result.append((raw_estimates[j - 1], biases[j - 1])) + elif raw_estimates[j] == x: + result.append((raw_estimates[j], biases[j])) + else: + # Найти 6 ближайших соседей. Вычислить среднее арифметическое. + + # 6 точек слева x [j-6 j-5 j-4 j-3 j-2 j-1] + + begin = max(j - 6, 0) - 1 + end = j - 1 + + T = [] + for k in range(end, begin, -1): + T.append(x - raw_estimates[k]) + + # 6 точек справа x [j j+1 j+2 j+3 j+4 j+5] + + begin = j + end = min(j + 5, len(raw_estimates) - 1) + 1 + + U = [] + for k in range(begin, end): + U.append(raw_estimates[k] - x) + + # Сливаем расстояния. + + V = [] + + lim = min(len(T), len(U)) + k1 = 0 + k2 = 0 + + while k1 < lim and k2 < lim: + if T[k1] == U[k2]: + V.append(j - k1 - 1) + V.append(j + k2) + k1 = k1 + 1 + k2 = k2 + 1 + elif T[k1] < U[k2]: + V.append(j - k1 - 1) + k1 = k1 + 1 + else: + V.append(j + k2) + k2 = k2 + 1 + + if k1 < len(T): + while k1 < len(T): + V.append(j - k1 - 1) + k1 = k1 + 1 + elif k2 < len(U): + while k2 < len(U): + V.append(j + k2) + k2 = k2 + 1 + + # Выбираем 6 ближайших точек. + # Вычисляем средние. + + begin = 0 + end = min(len(V), 6) + + sum = 0 + bias = 0 + for k in range(begin, end): + sum += raw_estimates[V[k]] + bias += biases[V[k]] + sum /= float(end) + bias /= float(end) + + result.append((sum, bias)) + + # Пропустить последовательные результаты, чьи оценки одинаковые. + final_result = [] + last = -1 + for entry in result: + if entry[0] != last: + final_result.append((entry[0], entry[1])) + last = entry[0] + + return final_result + +def dump_arrays(stats): + + print("Size of each array: {0}\n".format(len(stats))) + + is_first = True + sep = '' + + print("// For UniqCombinedBiasData::getRawEstimates():") + print("{") + for row in stats: + print("\t{0}{1}".format(sep, row[0])) + if is_first == True: + is_first = False + sep = "," + print("}") + + is_first = True + sep = "" + + print("\n// For UniqCombinedBiasData::getBiases():") + print("{") + for row in stats: + print("\t{0}{1}".format(sep, row[1])) + if is_first == True: + is_first = False + sep = "," + print("}") + +def start(): + parser = argparse.ArgumentParser(description = "Generate bias correction tables.") + parser.add_argument("-x", "--host", default="127.0.0.1", help="clickhouse host name"); + parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port"); + parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port"); + parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations"); + parser.add_argument("-s", "--generated", type=int, default=700000, help="number of generated values"); + parser.add_argument("-g", "--samples", type=int, default=200, help="number of sampled values"); + args = parser.parse_args() + + stats = [] + + for i in range(0, args.iterations): + print(i + 1) + sys.stdout.flush() + generate_data_source(args.host, str(args.port), str(args.http_port), 0, args.generated, 1000) + output = perform_query(args.host, str(args.port)) + data = parse_clickhouse_response(output) + stats = accumulate_data(stats, data) + + result = generate_raw_result(stats, args.iterations) + sample = generate_sample(result[0], result[1], args.samples) + dump_arrays(sample) + +if __name__ == "__main__": start() diff --git a/dbms/scripts/linear-counting-threshold.py b/dbms/scripts/linear-counting-threshold.py new file mode 100755 index 00000000000..eac57d50775 --- /dev/null +++ b/dbms/scripts/linear-counting-threshold.py @@ -0,0 +1,143 @@ +#!/usr/bin/python3.4 +# -*- coding: utf-8 -*- + +import sys +import argparse +import tempfile +import random +import subprocess +import bisect +from copy import deepcopy + +# http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/ +class UniqueRandomGenerator: + prime = 4294967291 + + def __init__(self, seed_base, seed_offset): + self.index = self.permutePQR(self.permutePQR(seed_base) + 0x682f0161) + self.intermediate_offset = self.permutePQR(self.permutePQR(seed_offset) + 0x46790905) + + def next(self): + val = self.permutePQR((self.permutePQR(self.index) + self.intermediate_offset) ^ 0x5bf03635) + self.index = self.index + 1 + return val + + def permutePQR(self, x): + if x >=self.prime: + return x + else: + residue = (x * x) % self.prime + if x <= self.prime/2: + return residue + else: + return self.prime - residue + +def generate_data_source(host, port, http_port, begin, end, count): + chunk_size = round((end - begin) / float(count)) + used_values = 0 + + cur_count = 0 + next_size = 0 + + sup = 32768 + n1 = random.randrange(0, sup) + n2 = random.randrange(0, sup) + urng = UniqueRandomGenerator(n1, n2) + + with tempfile.TemporaryDirectory() as tmp_dir: + filename = tmp_dir + '/table.txt' + file_handle = open(filename, 'w+b') + + while cur_count < count: + next_size += chunk_size + + while used_values < next_size: + h = urng.next() + used_values = used_values + 1 + outstr = str(h) + "\t" + str(cur_count) + "\n"; + file_handle.write(bytes(outstr, 'UTF-8')); + + cur_count = cur_count + 1 + + file_handle.close() + + query = 'DROP TABLE IF EXISTS data_source' + subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) + query = 'CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog' + subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) + + cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE) + subprocess.check_output(("POST", "http://localhost:{0}/?query=INSERT INTO data_source FORMAT TabSeparated".format(http_port)), stdin=cat.stdout) + cat.wait() + +def perform_query(host, port): + query = "SELECT runningAccumulate(uniqExactState(UserID)) AS exact, " + query += "runningAccumulate(uniqCombinedRawState(UserID)) AS raw, " + query += "runningAccumulate(uniqCombinedLinearCountingState(UserID)) AS linear_counting, " + query += "runningAccumulate(uniqCombinedBiasCorrectedState(UserID)) AS bias_corrected " + query += "FROM data_source GROUP BY KeyID" + return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query]) + +def parse_clickhouse_response(output): + parsed = [] + lines = output.decode().split("\n") + for cur_line in lines: + rows = cur_line.split("\t") + if len(rows) == 4: + parsed.append([float(rows[0]), float(rows[1]), float(rows[2]), float(rows[3])]) + return parsed + +def accumulate_data(stats, data): + if not stats: + stats = deepcopy(data) + else: + for row1, row2 in zip(stats, data): + row1[1] += row2[1]; + row1[2] += row2[2]; + row1[3] += row2[3]; + return stats + +def dump_graphs(stats, count): + fh1 = open("raw_graph.txt", "w+b") + fh2 = open("linear_counting_graph.txt", "w+b") + fh3 = open("bias_corrected_graph.txt", "w+b") + + expected_tab = [] + bias_tab = [] + for row in stats: + exact = row[0] + raw = row[1] / count; + linear_counting = row[2] / count; + bias_corrected = row[3] / count; + + outstr = "{0}\t{1}\n".format(exact, abs(raw - exact) / exact) + fh1.write(bytes(outstr, 'UTF-8')) + + outstr = "{0}\t{1}\n".format(exact, abs(linear_counting - exact) / exact) + fh2.write(bytes(outstr, 'UTF-8')) + + outstr = "{0}\t{1}\n".format(exact, abs(bias_corrected - exact) / exact) + fh3.write(bytes(outstr, 'UTF-8')) + +def start(): + parser = argparse.ArgumentParser(description = "Generate bias correction tables.") + parser.add_argument("-x", "--host", default="127.0.0.1", help="clickhouse host name"); + parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port"); + parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port"); + parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations"); + parser.add_argument("-s", "--generated", type=int, default=700000, help="number of generated values"); + args = parser.parse_args() + + stats = [] + + for i in range(0, args.iterations): + print(i + 1) + sys.stdout.flush() + generate_data_source(args.host, str(args.port), str(args.http_port), 0, args.generated, 1000) + output = perform_query(args.host, str(args.port)) + data = parse_clickhouse_response(output) + stats = accumulate_data(stats, data) + + dump_graphs(stats, args.iterations) + +if __name__ == "__main__": start() diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index c5f503da3f0..5a131098b9a 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -351,6 +351,72 @@ AggregateFunctionPtr AggregateFunctionFactory::get(const String & name, const Da else throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } + else if (name == "uniqCombinedRaw") + { + if (argument_types.size() != 1) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const IDataType & argument_type = *argument_types[0]; + + AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); + + if (res) + return res; + else if (typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else if (typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else + throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + else if (name == "uniqCombinedLinearCounting") + { + if (argument_types.size() != 1) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const IDataType & argument_type = *argument_types[0]; + + AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); + + if (res) + return res; + else if (typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else if (typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else + throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + else if (name == "uniqCombinedBiasCorrected") + { + if (argument_types.size() != 1) + throw Exception("Incorrect number of arguments for aggregate function " + name, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const IDataType & argument_type = *argument_types[0]; + + AggregateFunctionPtr res = createWithNumericType(*argument_types[0]); + + if (res) + return res; + else if (typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else if (typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else if (typeid_cast(&argument_type) || typeid_cast(&argument_type)) + return new AggregateFunctionUniq>; + else + throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } else if (name == "uniqCombined") { if (argument_types.size() != 1) @@ -733,6 +799,9 @@ const AggregateFunctionFactory::FunctionNames & AggregateFunctionFactory::getFun "uniq", "uniqHLL12", "uniqExact", + "uniqCombinedRaw", + "uniqCombinedLinearCounting", + "uniqCombinedBiasCorrected", "uniqCombined", "uniqUpTo", "groupArray", diff --git a/dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp b/dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp new file mode 100644 index 00000000000..faf6d452259 --- /dev/null +++ b/dbms/src/AggregateFunctions/UniqCombinedBiasData.cpp @@ -0,0 +1,390 @@ +#include + +namespace DB +{ + +namespace +{ + +const UniqCombinedBiasData::InterpolatedData raw_estimates = +{ + 700.0 + ,3850.0 + ,7350.0 + ,10850.0 + ,14350.0 + ,89003.5714 + ,103764.30343333333 + ,105572.1915 + ,109252.46533333334 + ,112638.20573333332 + ,116094.29566666669 + ,119619.81926666666 + ,123214.92233333334 + ,126469.06656666666 + ,130196.15093333334 + ,133566.85673333335 + ,136991.63890000002 + ,140470.0118666667 + ,144000.91686666667 + ,147585.44463333333 + ,151222.7466 + ,154447.75893333333 + ,158181.68399999998 + ,161492.41386666667 + ,164840.6352 + ,168713.9904 + ,172143.82656666666 + ,175611.2078 + ,179116.94873333335 + ,182658.0355 + ,186236.36723333332 + ,189332.1009 + ,192976.1847 + ,196654.62706666664 + ,199835.39103333335 + ,203575.92429999998 + ,206808.87086666666 + ,210611.72886666664 + ,213896.25913333334 + ,217759.63066666664 + ,221096.10933333333 + ,224456.31466666667 + ,227839.0366333333 + ,231242.72576666667 + ,235239.98256666667 + ,238688.95070000002 + ,242158.17593333332 + ,245649.42926666664 + ,249158.9859666667 + ,252689.67179999998 + ,256241.95376666667 + ,259214.9391666667 + ,262798.3925666667 + ,266399.8345666667 + ,270018.35863333335 + ,273653.1149 + ,276696.7119 + ,280366.51476666663 + ,284051.95540000004 + ,287133.5254333333 + ,290847.31173333334 + ,294579.5226 + ,297698.64109999995 + ,301454.39253333333 + ,305223.59123333334 + ,308375.3184666667 + ,312170.06 + ,315342.02996666665 + ,319162.8188666667 + ,322356.3565666666 + ,326199.5866 + ,329412.83396666666 + ,332634.3235666667 + ,336510.7596333333 + ,339747.7330333333 + ,343643.0385666667 + ,346896.77420000004 + ,350157.6729666667 + ,354079.3932333334 + ,357354.5196333334 + ,360638.3034333333 + ,364588.47873333335 + ,367886.05706666666 + ,371189.98006666667 + ,375161.95876666665 + ,378478.6737666666 + ,381801.6619 + ,385130.9645 + ,389131.7460333333 + ,392471.6233333333 + ,395817.1175 + ,399165.1003333333 + ,402518.7819333333 + ,406549.7624333333 + ,409916.016 + ,413289.0218666666 + ,416661.9977333333 + ,420040.4257333334 + ,424099.3186333333 + ,427485.4292000001 + ,430876.4814666666 + ,434269.4718 + ,437665.82826666674 + ,441066.7185 + ,444469.97226666665 + ,448561.9376666667 + ,451974.73750000005 + ,455389.1112 + ,458808.5816666667 + ,462230.8184666667 + ,465656.9889 + ,469081.3269 + ,472512.4878 + ,475944.4204333333 + ,480065.7132666667 + ,483502.04110000003 + ,486939.5075666667 + ,490379.7868333334 + ,493818.5365333333 + ,497259.08013333334 + ,500705.3513 + ,504155.6234666666 + ,507606.65499999997 + ,511060.7448666667 + ,514517.4004 + ,517973.35829999996 + ,521431.3761666666 + ,524891.7097333333 + ,529044.7593 + ,532507.0878999999 + ,535971.5070333333 + ,539436.2416999999 + ,542903.1470333333 + ,546370.3423 + ,549837.6947999999 + ,553307.0003666667 + ,556775.3770333333 + ,560247.6308333334 + ,563721.0700333334 + ,567196.7586333333 + ,570669.8439666666 + ,574146.018 + ,577622.2794666667 + ,581098.3862333334 + ,584575.8826666666 + ,588055.1468000001 + ,591538.0234 + ,595018.0103000001 + ,598504.5469333333 + ,601992.5697666666 + ,605475.5452 + ,608959.4645 + ,612444.0261 + ,615929.6436 + ,619412.3877333334 + ,622903.4263999999 + ,626391.3657333333 + ,629876.7359333333 + ,633364.2825999999 + ,636855.2673666667 + ,640344.4321000001 + ,643836.5543666667 + ,647327.3073999999 + ,650818.3525666667 + ,654312.2421666667 + ,657807.0899666668 + ,661301.4443666666 + ,664794.1040333334 + ,668288.1969666666 + ,671781.0196666667 + ,675272.7522333333 + ,678766.9045999999 + ,682259.3583666667 + ,685747.8148333334 + ,689238.7994666666 + ,692732.0478333334 + ,696224.6407 + ,700069.9224 +}; + +const UniqCombinedBiasData::InterpolatedData biases = +{ + 0.0 + ,0.0 + ,0.0 + ,0.0 + ,0.0 + ,71153.5714 + ,85214.30343333333 + ,83522.1915 + ,80202.46533333334 + ,77288.20573333332 + ,74444.29566666667 + ,71669.81926666667 + ,68964.92233333334 + ,66619.06656666666 + ,64046.15093333333 + ,61816.85673333333 + ,59641.6389 + ,57520.01186666667 + ,55450.91686666667 + ,53435.44463333334 + ,51472.74659999999 + ,49797.75893333333 + ,47931.68399999999 + ,46342.41386666667 + ,44790.6352 + ,43063.9904 + ,41593.82656666667 + ,40161.2078 + ,38766.94873333333 + ,37408.035500000005 + ,36086.36723333333 + ,34982.1009 + ,33726.184700000005 + ,32504.627066666664 + ,31485.391033333333 + ,30325.924299999995 + ,29358.870866666668 + ,28261.72886666667 + ,27346.259133333337 + ,26309.630666666668 + ,25446.109333333337 + ,24606.31466666666 + ,23789.036633333333 + ,22992.725766666666 + ,22089.98256666667 + ,21338.9507 + ,20608.175933333332 + ,19899.429266666673 + ,19208.985966666663 + ,18539.6718 + ,17891.95376666667 + ,17364.939166666667 + ,16748.392566666666 + ,16149.834566666666 + ,15568.358633333331 + ,15003.114899999995 + ,14546.711900000004 + ,14016.51476666668 + ,13501.955399999997 + ,13083.52543333332 + ,12597.311733333336 + ,12129.522600000006 + ,11748.641100000008 + ,11304.392533333332 + ,10873.59123333334 + ,10525.318466666678 + ,10120.059999999998 + ,9792.029966666674 + ,9412.818866666668 + ,9106.356566666664 + ,8749.58660000001 + ,8462.833966666678 + ,8184.323566666659 + ,7860.759633333325 + ,7597.733033333323 + ,7293.038566666665 + ,7046.774200000004 + ,6807.672966666675 + ,6529.393233333336 + ,6304.519633333344 + ,6088.30343333332 + ,5838.4787333333325 + ,5636.057066666661 + ,5439.980066666671 + ,5211.958766666658 + ,5028.673766666664 + ,4851.661899999996 + ,4680.964499999992 + ,4481.746033333319 + ,4321.623333333322 + ,4167.117500000012 + ,4015.1003333333356 + ,3868.781933333337 + ,3699.762433333332 + ,3566.0159999999937 + ,3439.021866666648 + ,3311.9977333333422 + ,3190.4257333333276 + ,3049.3186333333238 + ,2935.4291999999937 + ,2826.4814666666593 + ,2719.4717999999993 + ,2615.8282666666782 + ,2516.7184999999977 + ,2419.972266666669 + ,2311.9376666666744 + ,2224.7374999999884 + ,2139.1111999999944 + ,2058.581666666665 + ,1980.8184666666687 + ,1906.9888999999966 + ,1831.3268999999952 + ,1762.4878000000026 + ,1694.420433333328 + ,1615.7132666666682 + ,1552.0410999999924 + ,1489.507566666677 + ,1429.7868333333365 + ,1368.536533333332 + ,1309.0801333333268 + ,1255.35129999999 + ,1205.6234666666617 + ,1156.6549999999988 + ,1110.744866666675 + ,1067.4004000000034 + ,1023.3583000000023 + ,981.3761666666638 + ,941.7097333333513 + ,894.7593000000148 + ,857.0879000000035 + ,821.5070333333375 + ,786.2416999999745 + ,753.1470333333127 + ,720.3422999999797 + ,687.6947999999975 + ,657.0003666666647 + ,625.3770333333329 + ,597.6308333333387 + ,571.0700333333225 + ,546.7586333333165 + ,519.8439666666478 + ,496.0180000000012 + ,472.2794666666693 + ,448.386233333343 + ,425.8826666666816 + ,405.1468000000071 + ,388.0233999999861 + ,368.01030000002356 + ,354.54693333333125 + ,342.5697666666626 + ,325.5452000000126 + ,309.4644999999825 + ,294.0261000000173 + ,279.64360000001034 + ,262.38773333333666 + ,253.42639999999665 + ,241.36573333333945 + ,226.7359333333443 + ,214.28259999999622 + ,205.26736666667662 + ,194.43210000001514 + ,186.55436666666841 + ,177.30740000001 + ,168.35256666666828 + ,162.24216666668266 + ,157.0899666666713 + ,151.44436666666297 + ,144.1040333333464 + ,138.19696666668946 + ,131.01966666666945 + ,122.7522333333424 + ,116.90459999998954 + ,109.35836666667213 + ,97.81483333332774 + ,88.7994666666491 + ,82.04783333333519 + ,74.64070000000841 + ,69.92240000003949 +}; + +} + +double UniqCombinedBiasData::getThreshold() +{ + return 176000; +} + +const UniqCombinedBiasData::InterpolatedData & UniqCombinedBiasData::getRawEstimates() +{ + return raw_estimates; +} + +const UniqCombinedBiasData::InterpolatedData & UniqCombinedBiasData::getBiases() +{ + return biases; +} + +} diff --git a/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.reference b/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.reference index d66effa9fb1..b82ec5b6e3a 100644 --- a/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.reference +++ b/dbms/tests/queries/0_stateless/00212_aggregate_function_uniq.reference @@ -239,25 +239,25 @@ 31 162 35 162 36 162 -0 54226 -1 54034 -3 54016 -6 53982 -7 54076 -9 54218 -10 27075 -11 54093 -13 54108 -14 54096 -17 54294 -19 54070 -20 54028 -21 54170 -22 54106 -26 54103 -31 54050 -35 54130 -36 53868 +0 53901 +1 54056 +3 53999 +6 54129 +7 54213 +9 53853 +10 26975 +11 54064 +13 53963 +14 53997 +17 54129 +19 53923 +20 53958 +21 54117 +22 54150 +26 54047 +31 53948 +35 53931 +36 53982 0.125 1 0.5 1 0.05 1 @@ -291,25 +291,25 @@ 0.043 162 0.037 162 0.071 162 -0.045 54170 -0.125 54076 -0.5 54034 -0.05 54070 -0.143 53982 -0.091 27075 -0.056 54294 -0.048 54028 -0.083 54093 -0.25 54016 -1 54226 -0.1 54218 -0.028 54130 -0.027 53868 -0.031 54050 -0.067 54096 -0.043 54106 -0.037 54103 -0.071 54108 +0.045 54117 +0.125 54213 +0.5 54056 +0.05 53923 +0.143 54129 +0.091 26975 +0.056 54129 +0.048 53958 +0.083 54064 +0.25 53999 +1 53901 +0.1 53853 +0.028 53931 +0.027 53982 +0.031 53948 +0.067 53997 +0.043 54150 +0.037 54047 +0.071 53963 0.5 1 0.05 1 0.25 1 @@ -343,25 +343,25 @@ 0.037 162 0.1 163 1 162 -0.5 54034 -0.05 54070 -0.25 54016 -0.048 54028 -0.091 27075 -0.043 54106 -0.071 54108 -0.083 54093 -0.125 54076 -0.031 54050 -0.143 53982 -0.028 54130 -0.067 54096 -0.045 54170 -0.027 53868 -0.056 54294 -0.037 54103 -0.1 54218 -1 54226 +0.5 54056 +0.05 53923 +0.25 53999 +0.048 53958 +0.091 26975 +0.043 54150 +0.071 53963 +0.083 54064 +0.125 54213 +0.031 53948 +0.143 54129 +0.028 53931 +0.067 53997 +0.045 54117 +0.027 53982 +0.056 54129 +0.037 54047 +0.1 53853 +1 53901 1 1 3 1 6 1 From 6b5b1f2723b7729ccbaba07d0d7335ea8bd49e29 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Tue, 25 Aug 2015 10:50:01 +0300 Subject: [PATCH 17/50] optimistic codepath for bitTest(All|Any) when all positions are constant [#METR-17427] --- dbms/include/DB/Functions/FunctionsCoding.h | 89 ++++++++++++++++----- 1 file changed, 67 insertions(+), 22 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index 1d297bea127..1570c1002eb 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -1635,7 +1635,8 @@ private: if (const auto value_col = typeid_cast *>(value_col_untyped)) { const auto size = value_col->size(); - const auto & mask = createMask(size, block, arguments); + bool is_const; + const auto mask = createConstMask(size, block, arguments, is_const); const auto & val = value_col->getData(); const auto out_col = new ColumnVector(size); @@ -1644,25 +1645,46 @@ private: auto & out = out_col->getData(); - for (const auto i : ext::range(0, size)) - out[i] = Impl::combine(val[i], mask[i]); + if (is_const) + { + for (const auto i : ext::range(0, size)) + out[i] = Impl::combine(val[i], mask); + } + else + { + const auto mask = createMask(size, block, arguments); + + for (const auto i : ext::range(0, size)) + out[i] = Impl::combine(val[i], mask[i]); + } return true; } else if (const auto value_col = typeid_cast *>(value_col_untyped)) { const auto size = value_col->size(); - const auto & mask = createMask(size, block, arguments); - const auto & val = value_col->getData(); + bool is_const; + const auto mask = createConstMask(size, block, arguments, is_const); + const auto val = value_col->getData(); - const auto out_col = new ColumnVector(size); - ColumnPtr out_col_ptr{out_col}; - block.getByPosition(result).column = out_col_ptr; + if (is_const) + { + block.getByPosition(result).column = new ColumnConst{ + size, Impl::combine(val, mask) + }; + } + else + { + const auto mask = createMask(size, block, arguments); + const auto out_col = new ColumnVector(size); + ColumnPtr out_col_ptr{out_col}; + block.getByPosition(result).column = out_col_ptr; - auto & out = out_col->getData(); + auto & out = out_col->getData(); - for (const auto i : ext::range(0, size)) - out[i] = Impl::combine(val, mask[i]); + for (const auto i : ext::range(0, size)) + out[i] = Impl::combine(val, mask[i]); + } return true; } @@ -1670,26 +1692,49 @@ private: return false; } - template - PODArray createMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments) + template + ValueType createConstMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments, bool & is_const) { - PODArray mask(size, T{}); + is_const = true; + ValueType mask{}; for (const auto i : ext::range(1, arguments.size())) - addToMask(mask, block.getByPosition(arguments[i]).column.get()); + { + const auto pos_col = block.getByPosition(arguments[i]).column.get(); + + if (pos_col->isConst()) + { + const auto pos = static_cast *>(pos_col)->getData(); + mask = mask | 1 << pos; + } + else + { + is_const = false; + return {}; + } + } return mask; } template - void addToMask(PODArray & mask, const IColumn * const pos_col) + PODArray createMask(const std::size_t size, const Block & block, const ColumnNumbers & arguments) { - if (!addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col) && - !addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col)) - throw Exception{ - "Illegal column " + pos_col->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_COLUMN - }; + PODArray mask(size, ValueType{}); + + for (const auto i : ext::range(1, arguments.size())) + { + const auto pos_col = block.getByPosition(arguments[i]).column.get(); + + if (!addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col) && + !addToMaskImpl(mask, pos_col) && !addToMaskImpl(mask, pos_col)) + throw Exception{ + "Illegal column " + pos_col->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN + }; + } + + return mask; } template From e0e7ebfc0a06b5e278ee6305eb5a475ccd38a980 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Mon, 31 Aug 2015 18:32:01 +0300 Subject: [PATCH 18/50] dbms: Server: Coding style fix. [#METR-17276] --- dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h index 6a22bbb1a2b..1a444477adc 100644 --- a/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h +++ b/dbms/include/DB/AggregateFunctions/AggregateFunctionUniq.h @@ -229,7 +229,7 @@ struct OneAdder struct OneAdder>::value>::type> + std::is_same >::value>::type> { template static void addOne(Data & data, const IColumn & column, size_t row_num, From 77a0a22de638fb0a557ef2365a5d21978ecfc080 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Mon, 31 Aug 2015 19:49:09 +0300 Subject: [PATCH 19/50] dbms: existing_memory support for ReadBufferAIO.cpp --- dbms/include/DB/IO/BufferWithOwnMemory.h | 11 ++++++----- dbms/include/DB/IO/CachedCompressedReadBuffer.h | 9 ++++++++- dbms/include/DB/Storages/MergeTree/MergeTreeReader.h | 3 ++- dbms/src/IO/ReadBufferAIO.cpp | 4 +++- 4 files changed, 19 insertions(+), 8 deletions(-) diff --git a/dbms/include/DB/IO/BufferWithOwnMemory.h b/dbms/include/DB/IO/BufferWithOwnMemory.h index 6721c38a492..3f762a86642 100644 --- a/dbms/include/DB/IO/BufferWithOwnMemory.h +++ b/dbms/include/DB/IO/BufferWithOwnMemory.h @@ -66,15 +66,15 @@ struct Memory : boost::noncopyable, Allocator } else { - new_size = align(new_size); + new_size = align(new_size, alignment); + /// @todo pointer to void can be converted to pointer to any type with static_cast by ISO C++, reinterpret_cast has no advantages m_data = reinterpret_cast(Allocator::realloc(m_data, m_capacity, new_size, alignment)); m_capacity = new_size; m_size = m_capacity; } } -private: - size_t align(size_t value) const + static size_t align(const size_t value, const size_t alignment) { if (!alignment) return value; @@ -82,6 +82,7 @@ private: return (value + alignment - 1) / alignment * alignment; } +private: void alloc() { if (!m_capacity) @@ -93,8 +94,8 @@ private: ProfileEvents::increment(ProfileEvents::IOBufferAllocs); ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, m_capacity); - size_t new_capacity = align(m_capacity); - /// @todo void can be casted to any type with static_cast by ISO C++, reinterpret_cast has no advantages + size_t new_capacity = align(m_capacity, alignment); + /// @todo pointer to void can be converted to pointer to any type with static_cast by ISO C++, reinterpret_cast has no advantages m_data = reinterpret_cast(Allocator::alloc(new_capacity, alignment)); m_capacity = new_capacity; m_size = m_capacity; diff --git a/dbms/include/DB/IO/CachedCompressedReadBuffer.h b/dbms/include/DB/IO/CachedCompressedReadBuffer.h index d4fcebeb148..c3773731eb9 100644 --- a/dbms/include/DB/IO/CachedCompressedReadBuffer.h +++ b/dbms/include/DB/IO/CachedCompressedReadBuffer.h @@ -36,7 +36,14 @@ private: if (!file_in) { if (memory) - memory->resize(buf_size); + { + if (aio_threshold == 0 || estimated_size < aio_threshold) + memory->resize(buf_size); + else + memory->resize( + 2 * Memory::align(buf_size + DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE)); + } + file_in = createReadBufferFromFileBase( path, estimated_size, aio_threshold, buf_size, -1, memory ? &(*memory)[0] : nullptr); compressed_in = &*file_in; diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index cb7ce34cd1e..20f2489e175 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -221,7 +221,8 @@ private: if (aio_threshold == 0 || estimated_size < aio_threshold) memory.resize(buffer_size); else - memory.resize(2 * (buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE)); + memory.resize(2 * Memory::align(buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE, + DEFAULT_AIO_FILE_BLOCK_SIZE)); if (uncompressed_cache) { diff --git a/dbms/src/IO/ReadBufferAIO.cpp b/dbms/src/IO/ReadBufferAIO.cpp index 97dce3cfe5d..ed48f362e3b 100644 --- a/dbms/src/IO/ReadBufferAIO.cpp +++ b/dbms/src/IO/ReadBufferAIO.cpp @@ -13,7 +13,9 @@ namespace DB /// не влезают в основной буфер. ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, char * existing_memory_) : ReadBufferFromFileBase(buffer_size_ + DEFAULT_AIO_FILE_BLOCK_SIZE, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE), - fill_buffer(BufferWithOwnMemory(internalBuffer().size(), existing_memory_ + internalBuffer().size(), + fill_buffer(BufferWithOwnMemory(internalBuffer().size(), + existing_memory_ ? existing_memory_ + Memory::align(internalBuffer().size(), DEFAULT_AIO_FILE_BLOCK_SIZE) + : nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)), filename(filename_) { From 7b818fe243e808472f3fe2c3ec184aedfa1be2d8 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Mon, 31 Aug 2015 22:14:45 +0300 Subject: [PATCH 20/50] dbms: somehow fix memory corruption [#METR-16457] --- dbms/include/DB/Storages/MergeTree/MergeTreeReader.h | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index 20f2489e175..bd3612c5181 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -295,11 +295,13 @@ private: typedef std::map > FileStreams; + /** @todo placing buffers after streams (and causing buffers to be deleted bebfore streams in destructor) + causes memory corruption, investigate why */ + std::vector> buffers; String path; MergeTreeData::DataPartPtr data_part; String part_name; FileStreams streams; - std::vector> buffers; /// Запрашиваемые столбцы. NamesAndTypesList columns; From bc5007fc36ba2a81a6b45b9e9b0b26ff09efa994 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Mon, 31 Aug 2015 23:22:06 +0300 Subject: [PATCH 21/50] Merge --- dbms/include/DB/Storages/MergeTree/MergeTreeReader.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index bd3612c5181..33865427580 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -156,7 +156,6 @@ private: struct Stream { MarkCache::MappedPtr marks; - Memory & memory; ReadBuffer * data_buffer; std::unique_ptr cached_buffer; std::unique_ptr non_cached_buffer; @@ -169,7 +168,7 @@ private: 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, Memory & memory) - : memory(memory), path_prefix(path_prefix_) + : path_prefix(path_prefix_) { loadMarks(mark_cache); size_t max_mark_range = 0; From b2fa52394ba61dbdf3c5eef0f53531cf8df3855f Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Tue, 1 Sep 2015 15:24:38 +0300 Subject: [PATCH 22/50] dbms: support final with new code [#METR-16457] --- .../DB/Storages/MergeTree/MergeTreeReadPool.h | 13 ++++-- .../DB/Storages/MergeTree/MergeTreeReader.h | 5 ++- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 41 +++++++++++-------- 3 files changed, 35 insertions(+), 24 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index 8750f55b283..b4b35512397 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -40,8 +40,9 @@ public: MergeTreeReadPool( const std::size_t threads, const std::size_t sum_marks, const std::size_t min_marks_for_concurrent_read, RangesInDataParts parts, MergeTreeData & data, const ExpressionActionsPtr & prewhere_actions, - const String & prewhere_column_name, const bool check_columns, const Names & column_names) - : data{data}, column_names{column_names} + const String & prewhere_column_name, const bool check_columns, const Names & column_names, + const bool do_not_steal_tasks = false) + : data{data}, column_names{column_names}, do_not_steal_tasks{do_not_steal_tasks} { const auto per_part_sum_marks = fillPerPartInfo(parts, prewhere_actions, prewhere_column_name, check_columns); fillPerThreadInfo(threads, sum_marks, per_part_sum_marks, parts, min_marks_for_concurrent_read); @@ -57,8 +58,11 @@ public: if (remaining_thread_tasks.empty()) return nullptr; - const auto thread_idx = !threads_tasks[thread].sum_marks_in_parts.empty() ? thread : - *std::begin(remaining_thread_tasks); + const auto tasks_remaining_for_this_thread = !threads_tasks[thread].sum_marks_in_parts.empty(); + if (!tasks_remaining_for_this_thread && do_not_steal_tasks) + return nullptr; + + const auto thread_idx = tasks_remaining_for_this_thread ? thread : *std::begin(remaining_thread_tasks); auto & thread_tasks = threads_tasks[thread_idx]; auto & thread_task = thread_tasks.parts_and_ranges.back(); @@ -395,6 +399,7 @@ public: std::vector> per_part_columns_lock; MergeTreeData & data; Names column_names; + const bool do_not_steal_tasks; std::vector per_part_column_name_set; std::vector per_part_columns; std::vector per_part_pre_columns; diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index 33865427580..cae644371ba 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -294,8 +294,9 @@ private: typedef std::map > FileStreams; - /** @todo placing buffers after streams (and causing buffers to be deleted bebfore streams in destructor) - causes memory corruption, investigate why */ + /** buffers shall be deleted after streams because some streams may use existing_memory even inside destructor + * (ReadBufferAIO passes pointer to buffer to a syscall and waits for it's completion in destructor, thus there is + * a chance that system will write to memory after it has been freed */ std::vector> buffers; String path; MergeTreeData::DataPartPtr data_part; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c0c820d90f3..c1df0c93826 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -490,7 +490,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal const Settings & settings, const Context & context) { - size_t max_marks_to_use_cache = (settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity; + const size_t max_marks_to_use_cache = + (settings.merge_tree_max_rows_to_use_cache + data.index_granularity - 1) / data.index_granularity; + const size_t min_marks_for_read_task = + (settings.merge_tree_min_rows_for_concurrent_read + data.index_granularity - 1) / data.index_granularity; size_t sum_marks = 0; for (size_t i = 0; i < parts.size(); ++i) @@ -500,30 +503,32 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; + MergeTreeReadPoolPtr pool = std::make_shared( + parts.size(), sum_marks, min_marks_for_read_task, parts, data, prewhere_actions, prewhere_column, true, + column_names, true); + BlockInputStreams to_merge; - for (size_t part_index = 0; part_index < parts.size(); ++part_index) + for (const auto i : ext::range(0, parts.size())) { - RangesInDataPart & part = parts[part_index]; - - BlockInputStreamPtr source_stream = new MergeTreeBlockInputStream( - data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data, - part.data_part, part.ranges, use_uncompressed_cache, - prewhere_actions, prewhere_column, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size); - - for (const String & virt_column : virt_columns) - { - if (virt_column == "_part") - source_stream = new AddingConstColumnBlockInputStream( - source_stream, new DataTypeString, part.data_part->name, "_part"); - else if (virt_column == "_part_index") - source_stream = new AddingConstColumnBlockInputStream( - source_stream, new DataTypeUInt64, part.part_index_in_query, "_part_index"); - } + BlockInputStreamPtr source_stream{ + new MergeTreeThreadBlockInputStream{ + i, pool, min_marks_for_read_task, max_block_size, data, use_uncompressed_cache, prewhere_actions, + prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns + } + }; to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression())); } + /// Оценим общее количество строк - для прогресс-бара. + const std::size_t total_rows = data.index_granularity * sum_marks; + + /// Выставим приблизительное количество строк только для первого источника + static_cast(*to_merge.front()).setTotalRowsApprox(total_rows); + + LOG_TRACE(log, "Reading approx. " << total_rows); + BlockInputStreams res; if (to_merge.size() == 1) { From cbe8fb0b6790066e02af503ce39b6ccddf450f0c Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Tue, 1 Sep 2015 17:15:07 +0300 Subject: [PATCH 23/50] Merge --- .../DB/Storages/MergeTree/MergeTreeReader.h | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index cae644371ba..bdbb409241e 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -217,12 +217,6 @@ private: } } - if (aio_threshold == 0 || estimated_size < aio_threshold) - memory.resize(buffer_size); - else - memory.resize(2 * Memory::align(buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE, - DEFAULT_AIO_FILE_BLOCK_SIZE)); - if (uncompressed_cache) { cached_buffer = std::make_unique( @@ -231,6 +225,14 @@ private: } else { + if (aio_threshold == 0 || estimated_size < aio_threshold) + memory.resize(buffer_size); + else + memory.resize(2 * Memory::align(buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE, + DEFAULT_AIO_FILE_BLOCK_SIZE)); + + /** @todo CompressedReadBufferFromFile creates buffer for decompressed blocks, consider providing another + * instance of Memory type for it */ non_cached_buffer = std::make_unique( path_prefix + ".bin", estimated_size, aio_threshold, buffer_size, &memory[0]); data_buffer = non_cached_buffer.get(); @@ -325,7 +327,7 @@ private: return; const auto buffer_idx = streams.size(); - if (buffer_idx >= buffers.size()) + if (buffer_idx == buffers.size()) buffers.push_back(std::make_unique(0, DEFAULT_AIO_FILE_BLOCK_SIZE)); /// Для массивов используются отдельные потоки для размеров. From 1b394d9324c704d580c2d699afa4d11da23655ec Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Tue, 1 Sep 2015 18:08:37 +0300 Subject: [PATCH 24/50] dbms: DB::Memory::resize alloc instead of realloc when m_capacity == 0 [#METR-16457] --- dbms/include/DB/IO/BufferWithOwnMemory.h | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/dbms/include/DB/IO/BufferWithOwnMemory.h b/dbms/include/DB/IO/BufferWithOwnMemory.h index 3f762a86642..93f14173d30 100644 --- a/dbms/include/DB/IO/BufferWithOwnMemory.h +++ b/dbms/include/DB/IO/BufferWithOwnMemory.h @@ -59,7 +59,12 @@ struct Memory : boost::noncopyable, Allocator void resize(size_t new_size) { - if (new_size < m_capacity) + if (0 == m_capacity) + { + m_size = m_capacity = new_size; + alloc(); + } + else if (new_size < m_capacity) { m_size = new_size; return; From ef356f926797915189a83ff465a8b7638dfd11d2 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Tue, 1 Sep 2015 19:09:12 +0300 Subject: [PATCH 25/50] Merge --- dbms/include/DB/Interpreters/Settings.h | 3 + .../MergeTreeThreadBlockInputStream.h | 7 ++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 103 +++++++++++------- 3 files changed, 75 insertions(+), 38 deletions(-) diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 6e58df468cb..43f9f68fe69 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -119,7 +119,10 @@ struct Settings * (Чтобы большие запросы не вымывали кэш.) */ \ M(SettingUInt64, merge_tree_max_rows_to_use_cache, (1024 * 1024)) \ \ + /** Распределять чтение из MergeTree по потокам равномерно, обеспечивая стабильное среднее время исполнения каждого потока в пределах одного чтения. */ \ M(SettingBool, merge_tree_uniform_read_distribution, false) \ + /** Переиспользовать MergeTreeReader'ы после вычитывания подзадачи на чтение. */ \ + M(SettingBool, merge_tree_uniform_read_reuse_readers, false) \ \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \ diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 299c8ac1b1e..13dc0e9c43b 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -69,7 +69,14 @@ protected: injectVirtualColumns(res); if (task->mark_ranges.empty()) + { + if (0 == storage.context.getSettings().merge_tree_uniform_read_reuse_readers) + { + reader = {}; + pre_reader = {}; + } task = {}; + } } return res; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index c1df0c93826..1d77361674c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -369,14 +369,30 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - MergeTreeReadPoolPtr pool = std::make_shared( - threads, sum_marks, min_marks_for_concurrent_read, parts, data, prewhere_actions, prewhere_column, true, - column_names); - BlockInputStreams res; /// @todo remove old code - if (sum_marks > 0 && settings.merge_tree_uniform_read_distribution == 0) + if (sum_marks > 0 && settings.merge_tree_uniform_read_distribution == 1) + { + MergeTreeReadPoolPtr pool = std::make_shared( + threads, sum_marks, min_marks_for_concurrent_read, parts, data, prewhere_actions, prewhere_column, true, + column_names); + + for (std::size_t i = 0; i < threads; ++i) + res.emplace_back(new MergeTreeThreadBlockInputStream{ + i, pool, min_marks_for_concurrent_read, max_block_size, data, use_uncompressed_cache, prewhere_actions, + prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns + }); + + /// Оценим общее количество строк - для прогресс-бара. + const std::size_t total_rows = data.index_granularity * sum_marks; + + /// Выставим приблизительное количество строк только для первого источника + static_cast(*res.front()).setTotalRowsApprox(total_rows); + + LOG_TRACE(log, "Reading approx. " << total_rows); + } + else if (sum_marks > 0) { const size_t min_marks_per_thread = (sum_marks - 1) / threads + 1; @@ -458,22 +474,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( if (!parts.empty()) throw Exception("Couldn't spread marks among threads", ErrorCodes::LOGICAL_ERROR); } - else if (sum_marks > 0) - { - for (std::size_t i = 0; i < threads; ++i) - res.emplace_back(new MergeTreeThreadBlockInputStream{ - i, pool, min_marks_for_concurrent_read, max_block_size, data, use_uncompressed_cache, prewhere_actions, - prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns - }); - - /// Оценим общее количество строк - для прогресс-бара. - const std::size_t total_rows = data.index_granularity * sum_marks; - - /// Выставим приблизительное количество строк только для первого источника - static_cast(*res.front()).setTotalRowsApprox(total_rows); - - LOG_TRACE(log, "Reading approx. " << total_rows); - } return res; } @@ -503,31 +503,58 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal if (sum_marks > max_marks_to_use_cache) use_uncompressed_cache = false; - MergeTreeReadPoolPtr pool = std::make_shared( - parts.size(), sum_marks, min_marks_for_read_task, parts, data, prewhere_actions, prewhere_column, true, - column_names, true); - BlockInputStreams to_merge; - for (const auto i : ext::range(0, parts.size())) + if (settings.merge_tree_uniform_read_distribution == 1) { - BlockInputStreamPtr source_stream{ - new MergeTreeThreadBlockInputStream{ - i, pool, min_marks_for_read_task, max_block_size, data, use_uncompressed_cache, prewhere_actions, - prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns - } - }; + MergeTreeReadPoolPtr pool = std::make_shared( + parts.size(), sum_marks, min_marks_for_read_task, parts, data, prewhere_actions, prewhere_column, true, + column_names, true); - to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression())); + for (const auto i : ext::range(0, parts.size())) + { + BlockInputStreamPtr source_stream{ + new MergeTreeThreadBlockInputStream{ + i, pool, min_marks_for_read_task, max_block_size, data, use_uncompressed_cache, prewhere_actions, + prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns + } + }; + + to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression())); + } + + /// Оценим общее количество строк - для прогресс-бара. + const std::size_t total_rows = data.index_granularity * sum_marks; + + /// Выставим приблизительное количество строк только для первого источника + static_cast(*to_merge.front()).setTotalRowsApprox(total_rows); + + LOG_TRACE(log, "Reading approx. " << total_rows); } + else + { + for (size_t part_index = 0; part_index < parts.size(); ++part_index) + { + RangesInDataPart & part = parts[part_index]; - /// Оценим общее количество строк - для прогресс-бара. - const std::size_t total_rows = data.index_granularity * sum_marks; + BlockInputStreamPtr source_stream = new MergeTreeBlockInputStream( + data.getFullPath() + part.data_part->name + '/', max_block_size, column_names, data, + part.data_part, part.ranges, use_uncompressed_cache, + prewhere_actions, prewhere_column, true, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size); - /// Выставим приблизительное количество строк только для первого источника - static_cast(*to_merge.front()).setTotalRowsApprox(total_rows); + for (const String & virt_column : virt_columns) + { + if (virt_column == "_part") + source_stream = new AddingConstColumnBlockInputStream( + source_stream, new DataTypeString, part.data_part->name, "_part"); + else if (virt_column == "_part_index") + source_stream = new AddingConstColumnBlockInputStream( + source_stream, new DataTypeUInt64, part.part_index_in_query, "_part_index"); + } - LOG_TRACE(log, "Reading approx. " << total_rows); + to_merge.push_back(new ExpressionBlockInputStream(source_stream, data.getPrimaryExpression())); + } + } BlockInputStreams res; if (to_merge.size() == 1) From dab47ec36d7975616cf75a846e6ed233127a268f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2015 00:48:38 +0300 Subject: [PATCH 26/50] dbms: fixed error with StorageBuffer [#METR-17889]. --- dbms/src/Storages/StorageBuffer.cpp | 46 ++++++++++++++++------------- 1 file changed, 26 insertions(+), 20 deletions(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index a327df0e6d6..269ab8a3a35 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -149,8 +149,11 @@ static void appendBlock(const Block & from, Block & to) throw Exception("Cannot append block to another: different type of columns at index " + toString(column_no) + ". Block 1: " + from.dumpStructure() + ". Block 2: " + to.dumpStructure(), ErrorCodes::BLOCKS_HAS_DIFFERENT_STRUCTURE); - for (size_t row_no = 0; row_no < rows; ++row_no) - col_to.insertFrom(col_from, row_no); + if (col_to.empty()) + to.getByPosition(column_no).column = col_from.clone(); + else + for (size_t row_no = 0; row_no < rows; ++row_no) + col_to.insertFrom(col_from, row_no); } } @@ -243,24 +246,18 @@ private: buffer.data = sorted_block.cloneEmpty(); } - /// Если после вставки в буфер, ограничения будут превышены, то будем сбрасывать буфер. + /** Если после вставки в буфер, ограничения будут превышены, то будем сбрасывать буфер. + * Это также защищает от неограниченного потребления оперативки, так как в случае невозможности записать в таблицу, + * будет выкинуто исключение, а новые данные не будут добавлены в буфер. + */ if (storage.checkThresholds(buffer, time(0), sorted_block.rowsInFirstColumn(), sorted_block.bytes())) { - /// Вытащим из буфера блок, заменим буфер на пустой. После этого можно разблокировать mutex. - Block block_to_write; - buffer.data.swap(block_to_write); - buffer.first_write_time = 0; lock.unlock(); - - if (!storage.no_destination) - { - auto destination = storage.context.tryGetTable(storage.destination_database, storage.destination_table); - appendBlock(sorted_block, block_to_write); - storage.writeBlockToDestination(block_to_write, destination); - } + storage.flushBuffer(buffer, false); + lock.lock(); } - else - appendBlock(sorted_block, buffer.data); + + appendBlock(sorted_block, buffer.data); } }; @@ -331,8 +328,16 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds) { std::lock_guard lock(buffer.mutex); - if (check_thresholds && !checkThresholds(buffer, current_time)) - return; + if (check_thresholds) + { + if (!checkThresholds(buffer, current_time)) + return; + } + else + { + if (buffer.data.rowsInFirstColumn() == 0) + return; + } buffer.data.swap(block_to_write); buffer.first_write_time = 0; @@ -357,10 +362,11 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds) * Замечание: остаётся проблема - из-за того, что в разных попытках вставляются разные блоки, * теряется идемпотентность вставки в ReplicatedMergeTree. */ - appendBlock(block_to_write, buffer.data); - buffer.data.swap(block_to_write); + appendBlock(buffer.data, block_to_write); } + buffer.data.swap(block_to_write); + if (!buffer.first_write_time) buffer.first_write_time = current_time; From c5e0d759e5957ae723574d7884ac07582799d309 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2015 03:17:10 +0300 Subject: [PATCH 27/50] dbms: fixed build [#METR-17276]. --- dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h b/dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h index 59f809fb72e..eeb1bab2395 100644 --- a/dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h +++ b/dbms/include/DB/AggregateFunctions/UniqCombinedBiasData.h @@ -7,11 +7,11 @@ namespace DB /** Данные для HyperLogLogBiasEstimator в функции uniqCombined. * Схема разработки следующая: - * 1. Собрать clickhouse. + * 1. Собрать ClickHouse. * 2. Запустить скрипт src/dbms/scripts/gen-bias-data.py, который возвращает один массив для getRawEstimates() * и другой массив для getBiases(). * 3. Обновить массивы raw_estimates и biases. Также обновить размер массивов в InterpolatedData. - * 4. Собрать clickhouse. + * 4. Собрать ClickHouse. * 5. Запустить скрипт src/dbms/scripts/linear-counting-threshold.py, который создаёт 3 файла: * - raw_graph.txt (1-й столбец: настоящее количество уникальных значений; * 2-й столбец: относительная погрешность в случае HyperLogLog без применения каких-либо поправок) @@ -23,7 +23,7 @@ namespace DB * 7. Определить минимальное количество уникальных значений, при котором лучше исправить погрешность * с помощью её оценки (т.е. по алгоритму HyperLogLog++), чем применить алгоритм LinearCounting. * 7. Соответственно обновить константу в функции getThreshold() - * 8. Собрать clickhouse. + * 8. Собрать ClickHouse. */ struct UniqCombinedBiasData { From 044b70ff8ce443615e4d80d08d3e6876d56a8679 Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Thu, 27 Aug 2015 13:39:30 +0300 Subject: [PATCH 28/50] =?UTF-8?q?dbms::Allocator:=20support=20big=20alignm?= =?UTF-8?q?ents=20for=20=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88?= =?UTF-8?q?=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=20[#METR?= =?UTF-8?q?-17814]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- dbms/include/DB/Common/Allocator.h | 42 ++++++++++++++++++++---------- 1 file changed, 28 insertions(+), 14 deletions(-) diff --git a/dbms/include/DB/Common/Allocator.h b/dbms/include/DB/Common/Allocator.h index 524d88f010f..0ef3ec17838 100644 --- a/dbms/include/DB/Common/Allocator.h +++ b/dbms/include/DB/Common/Allocator.h @@ -36,16 +36,21 @@ public: if (size >= MMAP_THRESHOLD) { - if (alignment > MMAP_MIN_ALIGNMENT) - throw DB::Exception("Too large alignment: more than page size.", DB::ErrorCodes::BAD_ARGUMENTS); + if (alignment <= MMAP_MIN_ALIGNMENT) + { + buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == buf) + DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); - if (MAP_FAILED == buf) - DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - - /// См. комментарий в HashTableAllocator.h - if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE)) - DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + /// См. комментарий в HashTableAllocator.h + if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE)) + DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + } + else + { + /// если выравнивание больше, чем размер страницы используем posix_memalign + buf = allocateUsingPosixMemalign(size, alignment); + } } else { @@ -58,11 +63,7 @@ public: } else { - buf = nullptr; - int res = posix_memalign(&buf, alignment, size); - - if (0 != res) - DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + buf = allocateUsingPosixMemalign(size, alignment); } } @@ -121,4 +122,17 @@ public: return buf; } + +private: + void * allocateUsingPosixMemalign(size_t size, size_t alignment) + { + void * buf = nullptr; + + int res = posix_memalign(&buf, alignment, size); + + if (0 != res) + DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); + + return buf; + } }; From 7ef5c508d9d6172fc712d7ce3477a5a8b41c1dd3 Mon Sep 17 00:00:00 2001 From: Pavel Kartavyy Date: Tue, 1 Sep 2015 13:41:22 +0300 Subject: [PATCH 29/50] =?UTF-8?q?Revert=20"dbms::Allocator:=20support=20bi?= =?UTF-8?q?g=20alignments=20for=20=E2=96=88=E2=96=88=E2=96=88=E2=96=88?= =?UTF-8?q?=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88=E2=96=88?= =?UTF-8?q?=20[#METR-17814]"?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit 8d5d4b6b9265e2b5a103a6d713df81d7fe232745. --- dbms/include/DB/Common/Allocator.h | 42 ++++++++++-------------------- 1 file changed, 14 insertions(+), 28 deletions(-) diff --git a/dbms/include/DB/Common/Allocator.h b/dbms/include/DB/Common/Allocator.h index 0ef3ec17838..524d88f010f 100644 --- a/dbms/include/DB/Common/Allocator.h +++ b/dbms/include/DB/Common/Allocator.h @@ -36,21 +36,16 @@ public: if (size >= MMAP_THRESHOLD) { - if (alignment <= MMAP_MIN_ALIGNMENT) - { - buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); - if (MAP_FAILED == buf) - DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + if (alignment > MMAP_MIN_ALIGNMENT) + throw DB::Exception("Too large alignment: more than page size.", DB::ErrorCodes::BAD_ARGUMENTS); - /// См. комментарий в HashTableAllocator.h - if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE)) - DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); - } - else - { - /// если выравнивание больше, чем размер страницы используем posix_memalign - buf = allocateUsingPosixMemalign(size, alignment); - } + buf = mmap(NULL, size, PROT_READ | PROT_WRITE, MAP_PRIVATE | MAP_ANONYMOUS, -1, 0); + if (MAP_FAILED == buf) + DB::throwFromErrno("Allocator: Cannot mmap.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); + + /// См. комментарий в HashTableAllocator.h + if (size >= HUGE_PAGE_SIZE && 0 != madvise(buf, size, MADV_HUGEPAGE)) + DB::throwFromErrno("HashTableAllocator: Cannot madvise with MADV_HUGEPAGE.", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY); } else { @@ -63,7 +58,11 @@ public: } else { - buf = allocateUsingPosixMemalign(size, alignment); + buf = nullptr; + int res = posix_memalign(&buf, alignment, size); + + if (0 != res) + DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); } } @@ -122,17 +121,4 @@ public: return buf; } - -private: - void * allocateUsingPosixMemalign(size_t size, size_t alignment) - { - void * buf = nullptr; - - int res = posix_memalign(&buf, alignment, size); - - if (0 != res) - DB::throwFromErrno("Cannot allocate memory (posix_memalign)", DB::ErrorCodes::CANNOT_ALLOCATE_MEMORY, res); - - return buf; - } }; From 959d66cfa4f0ee25693a74dc76f79769990a71c5 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 2 Sep 2015 14:15:16 +0300 Subject: [PATCH 30/50] dbms: use actual settings [#METR-16457] --- .../MergeTree/MergeTreeThreadBlockInputStream.h | 11 ++++++----- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 13dc0e9c43b..1c1e9fd0801 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -19,12 +19,12 @@ public: const std::size_t thread, const MergeTreeReadPoolPtr & pool, const std::size_t min_marks_to_read, const std::size_t block_size, MergeTreeData & storage, const bool use_uncompressed_cache, const ExpressionActionsPtr & prewhere_actions, - const String & prewhere_column, const std::size_t min_bytes_to_use_direct_io, - const std::size_t max_read_buffer_size, const Names & virt_column_names) + const String & prewhere_column, const Settings & settings, const Names & virt_column_names) : thread{thread}, pool{pool}, min_marks_to_read{min_marks_to_read}, block_size{block_size}, storage{storage}, use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions}, - prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{min_bytes_to_use_direct_io}, - max_read_buffer_size{max_read_buffer_size}, virt_column_names{virt_column_names}, + prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{settings.min_bytes_to_use_direct_io}, + max_read_buffer_size{settings.max_read_buffer_size}, + reuse_readers(settings.merge_tree_uniform_read_reuse_readers), virt_column_names{virt_column_names}, log{&Logger::get("MergeTreeThreadBlockInputStream")} {} @@ -70,7 +70,7 @@ protected: if (task->mark_ranges.empty()) { - if (0 == storage.context.getSettings().merge_tree_uniform_read_reuse_readers) + if (!reuse_readers) { reader = {}; pre_reader = {}; @@ -325,6 +325,7 @@ private: const String prewhere_column; const std::size_t min_bytes_to_use_direct_io; const std::size_t max_read_buffer_size; + const bool reuse_readers; const Names virt_column_names; Logger * log; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 1d77361674c..88c7c2958d9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -381,7 +381,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( for (std::size_t i = 0; i < threads; ++i) res.emplace_back(new MergeTreeThreadBlockInputStream{ i, pool, min_marks_for_concurrent_read, max_block_size, data, use_uncompressed_cache, prewhere_actions, - prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns + prewhere_column, settings, virt_columns }); /// Оценим общее количество строк - для прогресс-бара. @@ -516,7 +516,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreadsFinal BlockInputStreamPtr source_stream{ new MergeTreeThreadBlockInputStream{ i, pool, min_marks_for_read_task, max_block_size, data, use_uncompressed_cache, prewhere_actions, - prewhere_column, settings.min_bytes_to_use_direct_io, settings.max_read_buffer_size, virt_columns + prewhere_column, settings, virt_columns } }; From 277fab46b9e5c9702f38c32c0735b18004349b10 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Wed, 2 Sep 2015 16:05:17 +0300 Subject: [PATCH 31/50] Merge --- dbms/include/DB/Functions/FunctionsCoding.h | 303 ++++++++++++++++-- dbms/src/Functions/FunctionsCoding.cpp | 2 + .../00076_ip_coding_functions.reference | 76 +++++ .../0_stateless/00076_ip_coding_functions.sql | 106 ++++++ 4 files changed, 453 insertions(+), 34 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsCoding.h b/dbms/include/DB/Functions/FunctionsCoding.h index 1d297bea127..a73195db2cb 100644 --- a/dbms/include/DB/Functions/FunctionsCoding.h +++ b/dbms/include/DB/Functions/FunctionsCoding.h @@ -42,31 +42,9 @@ namespace DB const auto ipv4_bytes_length = 4; const auto ipv6_bytes_length = 16; -class FunctionIPv6NumToString : public IFunction +class IPv6Format { -public: - static constexpr auto name = "IPv6NumToString"; - static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; } - - String getName() const { return name; } - - DataTypePtr getReturnType(const DataTypes & arguments) const - { - if (arguments.size() != 1) - throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " - + toString(arguments.size()) + ", should be 1.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const auto ptr = typeid_cast(arguments[0].get()); - if (!ptr || ptr->getN() != ipv6_bytes_length) - throw Exception("Illegal type " + arguments[0]->getName() + - " of argument of function " + getName() + - ", expected FixedString(" + toString(ipv6_bytes_length) + ")", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - return new DataTypeString; - } - +private: /// integer logarithm, return ceil(log(value, base)) (the smallest integer greater or equal than log(value, base) static constexpr uint32_t int_log(const uint32_t value, const uint32_t base, const bool carry = false) { @@ -99,23 +77,25 @@ public: } /// print IPv4 address as %u.%u.%u.%u - static void ipv4_format(const unsigned char * src, char *& dst) + static void ipv4_format(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count) { - constexpr auto size = sizeof(UInt32); + const auto limit = ipv4_bytes_length - zeroed_tail_bytes_count; - for (const auto i : ext::range(0, size)) + for (const auto i : ext::range(0, ipv4_bytes_length)) { - print_integer<10, UInt8>(dst, src[i]); + UInt8 byte = (i < limit) ? src[i] : 0; + print_integer<10, UInt8>(dst, byte); - if (i != size - 1) + if (i != ipv4_bytes_length - 1) *dst++ = '.'; } } +public: /** rewritten inet_ntop6 from http://svn.apache.org/repos/asf/apr/apr/trunk/network_io/unix/inet_pton.c * performs significantly faster than the reference implementation due to the absence of sprintf calls, * bounds checking, unnecessary string copying and length calculation */ - static const void ipv6_format(const unsigned char * src, char *& dst) + static const void apply(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_count = 0) { struct { int base, len; } best{-1}, cur{-1}; std::array words{}; @@ -123,7 +103,7 @@ public: /** Preprocess: * Copy the input (bytewise) array into a wordwise array. * Find the longest run of 0x00's in src[] for :: shorthanding. */ - for (const auto i : ext::range(0, ipv6_bytes_length)) + for (const auto i : ext::range(0, ipv6_bytes_length - zeroed_tail_bytes_count)) words[i / 2] |= src[i] << ((1 - (i % 2)) << 3); for (const auto i : ext::range(0, words.size())) @@ -172,7 +152,7 @@ public: /// Is this address an encapsulated IPv4? if (i == 6 && best.base == 0 && (best.len == 6 || (best.len == 5 && words[5] == 0xffffu))) { - ipv4_format(src + 12, dst); + ipv4_format(src + 12, dst, std::min(zeroed_tail_bytes_count, static_cast(ipv4_bytes_length))); break; } @@ -185,6 +165,32 @@ public: *dst++ = '\0'; } +}; + +class FunctionIPv6NumToString : public IFunction +{ +public: + static constexpr auto name = "IPv6NumToString"; + static IFunction * create(const Context & context) { return new FunctionIPv6NumToString; } + + String getName() const { return name; } + + DataTypePtr getReturnType(const DataTypes & arguments) const + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto ptr = typeid_cast(arguments[0].get()); + if (!ptr || ptr->getN() != ipv6_bytes_length) + throw Exception("Illegal type " + arguments[0]->getName() + + " of argument of function " + getName() + + ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return new DataTypeString; + } void execute(Block & block, const ColumnNumbers & arguments, const size_t result) { @@ -216,7 +222,7 @@ public: for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i) { - ipv6_format(&vec_in[offset], pos); + IPv6Format::apply(&vec_in[offset], pos); offsets_res[i] = pos - begin; } @@ -236,7 +242,7 @@ public: char buf[INET6_ADDRSTRLEN]; char * dst = buf; - ipv6_format(reinterpret_cast(data_in.data()), dst); + IPv6Format::apply(reinterpret_cast(data_in.data()), dst); block.getByPosition(result).column = new ColumnConstString{col_in->size(), buf}; } @@ -247,6 +253,172 @@ public: } }; +class FunctionCutIPv6 : public IFunction +{ +public: + static constexpr auto name = "cutIPv6"; + static IFunction * create(const Context & context) { return new FunctionCutIPv6; } + + String getName() const { return name; } + + DataTypePtr getReturnType(const DataTypes & arguments) const + { + if (arguments.size() != 3) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 3.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + const auto ptr = typeid_cast(arguments[0].get()); + if (!ptr || ptr->getN() != ipv6_bytes_length) + throw Exception("Illegal type " + arguments[0]->getName() + + " of argument 1 of function " + getName() + + ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!typeid_cast(arguments[1].get())) + throw Exception("Illegal type " + arguments[1]->getName() + + " of argument 2 of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + if (!typeid_cast(arguments[2].get())) + throw Exception("Illegal type " + arguments[2]->getName() + + " of argument 3 of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return new DataTypeString; + } + + void execute(Block & block, const ColumnNumbers & arguments, const size_t result) + { + const auto & col_name_type = block.getByPosition(arguments[0]); + const ColumnPtr & column = col_name_type.column; + + const auto & col_ipv6_zeroed_tail_bytes_type = block.getByPosition(arguments[1]); + const auto & col_ipv6_zeroed_tail_bytes = col_ipv6_zeroed_tail_bytes_type.column; + const auto & col_ipv4_zeroed_tail_bytes_type = block.getByPosition(arguments[2]); + const auto & col_ipv4_zeroed_tail_bytes = col_ipv4_zeroed_tail_bytes_type.column; + + if (const auto col_in = typeid_cast(column.get())) + { + if (col_in->getN() != ipv6_bytes_length) + throw Exception("Illegal type " + col_name_type.type->getName() + + " of column " + col_in->getName() + + " argument of function " + getName() + + ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto ipv6_zeroed_tail_bytes = typeid_cast *>(col_ipv6_zeroed_tail_bytes.get()); + if (!ipv6_zeroed_tail_bytes) + throw Exception("Illegal type " + col_ipv6_zeroed_tail_bytes_type.type->getName() + + " of argument 2 of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getData(); + if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length) + throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() + + " of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto ipv4_zeroed_tail_bytes = typeid_cast *>(col_ipv4_zeroed_tail_bytes.get()); + if (!ipv4_zeroed_tail_bytes) + throw Exception("Illegal type " + col_ipv4_zeroed_tail_bytes_type.type->getName() + + " of argument 3 of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getData(); + if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length) + throw Exception("Illegal value for argument 3 " + col_ipv4_zeroed_tail_bytes_type.type->getName() + + " of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto size = col_in->size(); + const auto & vec_in = col_in->getChars(); + + auto col_res = new ColumnString; + block.getByPosition(result).column = col_res; + + ColumnString::Chars_t & vec_res = col_res->getChars(); + ColumnString::Offsets_t & offsets_res = col_res->getOffsets(); + vec_res.resize(size * INET6_ADDRSTRLEN); + offsets_res.resize(size); + + auto begin = reinterpret_cast(&vec_res[0]); + auto pos = begin; + + for (size_t offset = 0, i = 0; offset < vec_in.size(); offset += ipv6_bytes_length, ++i) + { + const auto address = &vec_in[offset]; + UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count; + cutAddress(address, pos, zeroed_tail_bytes_count); + offsets_res[i] = pos - begin; + } + + vec_res.resize(pos - begin); + } + else if (const auto col_in = typeid_cast *>(column.get())) + { + const auto data_type_fixed_string = typeid_cast(col_in->getDataType().get()); + if (!data_type_fixed_string || data_type_fixed_string->getN() != ipv6_bytes_length) + throw Exception("Illegal type " + col_name_type.type->getName() + + " of column " + col_in->getName() + + " argument of function " + getName() + + ", expected FixedString(" + toString(ipv6_bytes_length) + ")", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto ipv6_zeroed_tail_bytes = typeid_cast *>(col_ipv6_zeroed_tail_bytes.get()); + if (!ipv6_zeroed_tail_bytes) + throw Exception("Illegal type " + col_ipv6_zeroed_tail_bytes_type.type->getName() + + " of argument 2 of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + UInt8 ipv6_zeroed_tail_bytes_count = ipv6_zeroed_tail_bytes->getData(); + if (ipv6_zeroed_tail_bytes_count > ipv6_bytes_length) + throw Exception("Illegal value for argument 2 " + col_ipv6_zeroed_tail_bytes_type.type->getName() + + " of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto ipv4_zeroed_tail_bytes = typeid_cast *>(col_ipv4_zeroed_tail_bytes.get()); + if (!ipv4_zeroed_tail_bytes) + throw Exception("Illegal type " + col_ipv4_zeroed_tail_bytes_type.type->getName() + + " of argument 3 of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + UInt8 ipv4_zeroed_tail_bytes_count = ipv4_zeroed_tail_bytes->getData(); + if (ipv4_zeroed_tail_bytes_count > ipv6_bytes_length) + throw Exception("Illegal value for argument 3 " + col_ipv6_zeroed_tail_bytes_type.type->getName() + + " of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + const auto & data_in = col_in->getData(); + + char buf[INET6_ADDRSTRLEN]; + char * dst = buf; + + const auto address = reinterpret_cast(data_in.data()); + UInt8 zeroed_tail_bytes_count = isIPv4Mapped(address) ? ipv4_zeroed_tail_bytes_count : ipv6_zeroed_tail_bytes_count; + cutAddress(address, dst, zeroed_tail_bytes_count); + + block.getByPosition(result).column = new ColumnConstString{col_in->size(), buf}; + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + +private: + bool isIPv4Mapped(const unsigned char * address) const + { + return (*reinterpret_cast(&address[0]) == 0) && + ((*reinterpret_cast(&address[8]) & 0x00000000FFFFFFFFull) == 0x00000000FFFF0000ull); + } + + void cutAddress(const unsigned char * address, char *& dst, UInt8 zeroed_tail_bytes_count) + { + IPv6Format::apply(address, dst, zeroed_tail_bytes_count); + } +}; + class FunctionIPv6StringToNum : public IFunction { public: @@ -767,6 +939,69 @@ public: } }; +class FunctionIPv4ToIPv6 : public IFunction +{ +public: + static constexpr auto name = "IPv4ToIPv6"; + static IFunction * create(const Context & context) { return new FunctionIPv4ToIPv6; } + + String getName() const { return name; } + + DataTypePtr getReturnType(const DataTypes & arguments) const + { + if (arguments.size() != 1) + throw Exception("Number of arguments for function " + getName() + " doesn't match: passed " + + toString(arguments.size()) + ", should be 1.", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + if (typeid_cast(arguments[0].get()) == nullptr) + throw Exception("Illegal type " + arguments[0]->getName() + + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + + return new DataTypeFixedString{16}; + } + + void execute(Block & block, const ColumnNumbers & arguments, const size_t result) + { + const auto & col_name_type = block.getByPosition(arguments[0]); + const ColumnPtr & column = col_name_type.column; + + if (const auto col_in = typeid_cast *>(column.get())) + { + const auto col_res = new ColumnFixedString{ipv6_bytes_length}; + block.getByPosition(result).column = col_res; + + auto & vec_res = col_res->getChars(); + vec_res.resize(col_in->size() * ipv6_bytes_length); + + const auto & vec_in = col_in->getData(); + + for (size_t out_offset = 0, i = 0; out_offset < vec_res.size(); out_offset += ipv6_bytes_length, ++i) + mapIPv4ToIPv6(vec_in[i], &vec_res[out_offset]); + } + else if (const auto col_in = typeid_cast *>(column.get())) + { + std::string buf; + buf.resize(ipv6_bytes_length); + mapIPv4ToIPv6(col_in->getData(), reinterpret_cast(&buf[0])); + + ColumnConstString * col_res = new ColumnConstString(ipv6_bytes_length, buf, + new DataTypeFixedString{ipv6_bytes_length}); + block.getByPosition(result).column = col_res; + } + else + throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() + + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_COLUMN); + } + +private: + void mapIPv4ToIPv6(UInt32 in, unsigned char * buf) const + { + *reinterpret_cast(&buf[0]) = 0; + *reinterpret_cast(&buf[8]) = 0x00000000FFFF0000ull | (static_cast(ntohl(in)) << 32); + } +}; class FunctionHex : public IFunction { diff --git a/dbms/src/Functions/FunctionsCoding.cpp b/dbms/src/Functions/FunctionsCoding.cpp index d213f0f6249..f2677c1bb74 100644 --- a/dbms/src/Functions/FunctionsCoding.cpp +++ b/dbms/src/Functions/FunctionsCoding.cpp @@ -8,10 +8,12 @@ void registerFunctionsCoding(FunctionFactory & factory) { factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference index 788f414fa87..129d52f3a73 100644 --- a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference +++ b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.reference @@ -16,3 +16,79 @@ 1 1 1 +00000000000000000000FFFF4D583737 +00000000000000000000FFFF4D583737 +00000000000000000000FFFF7F000001 +00000000000000000000FFFFC1FC110A +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f000 +2001:db8:ac10:fe01:feed:babe:cafe:0 +2001:db8:ac10:fe01:feed:babe:ca00:0 +2001:db8:ac10:fe01:feed:babe:: +2001:db8:ac10:fe01:feed:ba00:: +2001:db8:ac10:fe01:feed:: +2001:db8:ac10:fe01:fe00:: +2001:db8:ac10:fe01:: +2001:db8:ac10:fe00:: +2001:db8:ac10:: +2001:db8:ac00:: +2001:db8:: +2001:d00:: +2001:: +2000:: +:: +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +2001:db8:ac10:fe01:feed:babe:cafe:f00d +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.10 +::ffff:193.252.17.0 +::ffff:193.252.0.0 +::ffff:193.0.0.0 +::ffff:0.0.0.0 +::ff00:0:0 +:: +:: +:: +:: +:: +:: +:: +:: +:: +:: +:: +2001:db8:ac10:fe01:abad:babe:fa00:0 +2001:db8:ac10:fe01:dead:c0de:ca00:0 +2001:db8:ac10:fe01:feed:babe:ca00:0 +::ffff:77.0.0.0 +::ffff:127.0.0.0 +::ffff:193.0.0.0 diff --git a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql index 36bf6b3712b..56597f53357 100644 --- a/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql +++ b/dbms/tests/queries/0_stateless/00076_ip_coding_functions.sql @@ -17,3 +17,109 @@ select IPv6StringToNum('') == toFixedString(materialize(''), 16); select IPv6StringToNum(materialize('')) == toFixedString(materialize(''), 16); select IPv6StringToNum('not an ip string') == toFixedString(materialize(''), 16); select IPv6StringToNum(materialize('not an ip string')) == toFixedString(materialize(''), 16); + +/* IPv4ToIPv6 */ + +SELECT hex(IPv4ToIPv6(1297626935)); + +/* Тест с таблицей */ + +DROP TABLE IF EXISTS test.addresses; +CREATE TABLE test.addresses(addr UInt32) ENGINE = Memory; +INSERT INTO test.addresses(addr) VALUES (1297626935), (2130706433), (3254522122); +SELECT hex(IPv4ToIPv6(addr)) FROM test.addresses ORDER BY addr ASC; + +/* cutIPv6 */ + +/* Реальный IPv6-адрес */ + +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 0); + +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 1, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 2, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 3, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 4, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 5, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 6, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 7, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 8, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 9, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 10, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 11, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 12, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 13, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 14, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 15, 0); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 16, 0); + +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 1); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 2); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 3); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 4); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 5); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 6); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 7); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 8); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 9); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 10); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 11); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 12); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 13); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 14); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 15); +SELECT cutIPv6(IPv6StringToNum('2001:0DB8:AC10:FE01:FEED:BABE:CAFE:F00D'), 0, 16); + +/* IPv4-mapped IPv6-адрес */ + +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 0); + +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 1, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 2, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 3, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 4, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 5, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 6, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 7, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 8, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 9, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 10, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 11, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 12, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 13, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 14, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 15, 0); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 16, 0); + +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 1); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 2); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 3); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 4); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 5); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 6); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 7); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 8); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 9); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 10); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 11); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 12); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 13); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 14); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 15); +SELECT cutIPv6(toFixedString(unhex('00000000000000000000FFFFC1FC110A'), 16), 0, 16); + +/* Тест с таблицами */ + +/* Реальные IPv6-адреса */ + +DROP TABLE IF EXISTS test.addresses; +CREATE TABLE test.addresses(addr String) ENGINE = Memory; +INSERT INTO test.addresses(addr) VALUES ('20010DB8AC10FE01FEEDBABECAFEF00D'), ('20010DB8AC10FE01DEADC0DECAFED00D'), ('20010DB8AC10FE01ABADBABEFACEB00C'); +SELECT cutIPv6(toFixedString(unhex(addr), 16), 3, 0) FROM test.addresses ORDER BY addr ASC; + +/* IPv4-mapped IPv6-адреса */ + +DROP TABLE IF EXISTS test.addresses; +CREATE TABLE test.addresses(addr String) ENGINE = Memory; +INSERT INTO test.addresses(addr) VALUES ('00000000000000000000FFFFC1FC110A'), ('00000000000000000000FFFF4D583737'), ('00000000000000000000FFFF7F000001'); +SELECT cutIPv6(toFixedString(unhex(addr), 16), 0, 3) FROM test.addresses ORDER BY addr ASC; + From 70ec50316e879066d1a9ad635bcaf51a20ecd4d9 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 2 Sep 2015 17:48:50 +0300 Subject: [PATCH 32/50] Merge --- dbms/include/DB/IO/CachedCompressedReadBuffer.h | 15 ++++++++++++--- .../DB/Storages/MergeTree/MergeTreeReader.h | 13 ++++++++++--- 2 files changed, 22 insertions(+), 6 deletions(-) diff --git a/dbms/include/DB/IO/CachedCompressedReadBuffer.h b/dbms/include/DB/IO/CachedCompressedReadBuffer.h index c3773731eb9..a2067f0e4e3 100644 --- a/dbms/include/DB/IO/CachedCompressedReadBuffer.h +++ b/dbms/include/DB/IO/CachedCompressedReadBuffer.h @@ -37,11 +37,20 @@ private: { if (memory) { + auto & memory = *this->memory; + + const auto resize = [&memory] (const std::size_t size) { + const auto growth_factor = 1.6f; /// close to golden ratio + if (memory.m_capacity == 0) + memory.resize(size); + else if (memory.m_capacity < size) + memory.resize(growth_factor * size); + }; + if (aio_threshold == 0 || estimated_size < aio_threshold) - memory->resize(buf_size); + resize(buf_size); else - memory->resize( - 2 * Memory::align(buf_size + DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE)); + resize(2 * Memory::align(buf_size + DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE)); } file_in = createReadBufferFromFileBase( diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index bdbb409241e..f1105b47922 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -225,11 +225,18 @@ private: } else { + const auto resize = [&memory] (const std::size_t size) { + const auto growth_factor = 1.6f; /// close to golden ratio + if (memory.m_capacity == 0) + memory.resize(size); + else if (memory.m_capacity < size) + memory.resize(growth_factor * size); + }; + if (aio_threshold == 0 || estimated_size < aio_threshold) - memory.resize(buffer_size); + resize(buffer_size); else - memory.resize(2 * Memory::align(buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE, - DEFAULT_AIO_FILE_BLOCK_SIZE)); + resize(2 * Memory::align(buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE)); /** @todo CompressedReadBufferFromFile creates buffer for decompressed blocks, consider providing another * instance of Memory type for it */ From 288f46936a3a344202035eeabbd6e2a2d236c7f9 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 2 Sep 2015 18:48:10 +0300 Subject: [PATCH 33/50] Merge --- dbms/include/DB/Interpreters/Settings.h | 4 ++-- .../DB/Storages/MergeTree/MergeTreeReader.h | 24 +++++++++++++------ .../MergeTreeThreadBlockInputStream.h | 15 ++++-------- 3 files changed, 23 insertions(+), 20 deletions(-) diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 43f9f68fe69..2251b020dac 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -121,8 +121,8 @@ struct Settings \ /** Распределять чтение из MergeTree по потокам равномерно, обеспечивая стабильное среднее время исполнения каждого потока в пределах одного чтения. */ \ M(SettingBool, merge_tree_uniform_read_distribution, false) \ - /** Переиспользовать MergeTreeReader'ы после вычитывания подзадачи на чтение. */ \ - M(SettingBool, merge_tree_uniform_read_reuse_readers, false) \ + /** Переиспользовать буферы в MergeTreeReader::reconf() после выполнения подзадачи на чтение. */ \ + M(SettingBool, merge_tree_uniform_read_reuse_buffers, false) \ \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \ diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index f1105b47922..13903ace69e 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -31,9 +31,9 @@ public: const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MergeTreeData & storage_, const MarkRanges & all_mark_ranges, - size_t aio_threshold_, size_t max_read_buffer_size_) + size_t aio_threshold_, size_t max_read_buffer_size_, bool reuse_buffers = false) : uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_), storage(storage_), - aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_) + aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_), reuse_buffers{reuse_buffers} { reconf(path_, data_part, columns_, all_mark_ranges); } @@ -42,6 +42,13 @@ public: const String & path, const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns, const MarkRanges & all_mark_ranges) { + /// @todo should avg_value_size_hints be cleared on this->data_part != data_part? supposedly so + if (this->data_part != data_part) + avg_value_size_hints.clear(); + + if (!reuse_buffers) + buffers.clear(); + this->path = path; this->data_part = data_part; this->part_name = data_part->name; @@ -307,6 +314,7 @@ private: * (ReadBufferAIO passes pointer to buffer to a syscall and waits for it's completion in destructor, thus there is * a chance that system will write to memory after it has been freed */ std::vector> buffers; + std::map avg_value_size_hints; String path; MergeTreeData::DataPartPtr data_part; String part_name; @@ -322,6 +330,7 @@ private: MarkRanges all_mark_ranges; size_t aio_threshold; size_t max_read_buffer_size; + bool reuse_buffers; void addStream(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, size_t level = 0) { @@ -415,8 +424,9 @@ private: else { Stream & stream = *streams[name]; + double & avg_value_size_hint = avg_value_size_hints[name]; stream.seekToMark(from_mark); - type.deserializeBinary(column, *stream.data_buffer, max_rows_to_read, stream.avg_value_size_hint); + type.deserializeBinary(column, *stream.data_buffer, max_rows_to_read, avg_value_size_hint); /// Вычисление подсказки о среднем размере значения. size_t column_size = column.size(); @@ -425,10 +435,10 @@ private: double current_avg_value_size = static_cast(column.byteSize()) / column_size; /// Эвристика, чтобы при изменениях, значение avg_value_size_hint быстро росло, но медленно уменьшалось. - if (current_avg_value_size > stream.avg_value_size_hint) - stream.avg_value_size_hint = current_avg_value_size; - else if (current_avg_value_size * 2 < stream.avg_value_size_hint) - stream.avg_value_size_hint = (current_avg_value_size + stream.avg_value_size_hint * 3) / 4; + if (current_avg_value_size > avg_value_size_hint) + avg_value_size_hint = current_avg_value_size; + else if (current_avg_value_size * 2 < avg_value_size_hint) + avg_value_size_hint = (current_avg_value_size + avg_value_size_hint * 3) / 4; } } } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 1c1e9fd0801..0d6baf894e4 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -24,7 +24,7 @@ public: use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions}, prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{settings.min_bytes_to_use_direct_io}, max_read_buffer_size{settings.max_read_buffer_size}, - reuse_readers(settings.merge_tree_uniform_read_reuse_readers), virt_column_names{virt_column_names}, + reuse_buffers(settings.merge_tree_uniform_read_reuse_buffers), virt_column_names{virt_column_names}, log{&Logger::get("MergeTreeThreadBlockInputStream")} {} @@ -69,14 +69,7 @@ protected: injectVirtualColumns(res); if (task->mark_ranges.empty()) - { - if (!reuse_readers) - { - reader = {}; - pre_reader = {}; - } task = {}; - } } return res; @@ -108,13 +101,13 @@ private: reader = std::make_unique( path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), - storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); + storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size, reuse_buffers); if (prewhere_actions) pre_reader = std::make_unique( path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), storage, task->mark_ranges, min_bytes_to_use_direct_io, - max_read_buffer_size); + max_read_buffer_size, reuse_buffers); } else { @@ -325,7 +318,7 @@ private: const String prewhere_column; const std::size_t min_bytes_to_use_direct_io; const std::size_t max_read_buffer_size; - const bool reuse_readers; + const bool reuse_buffers; const Names virt_column_names; Logger * log; From 6b642e8749fcb222eb84267e5a6973139c361082 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 2 Sep 2015 19:09:32 +0300 Subject: [PATCH 34/50] dbms: round subtask size up to multiple of max_block_size [#METR-16457] --- .../MergeTree/MergeTreeThreadBlockInputStream.h | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index 0d6baf894e4..e79d36fe83f 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -20,8 +20,9 @@ public: const MergeTreeReadPoolPtr & pool, const std::size_t min_marks_to_read, const std::size_t block_size, MergeTreeData & storage, const bool use_uncompressed_cache, const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column, const Settings & settings, const Names & virt_column_names) - : thread{thread}, pool{pool}, min_marks_to_read{min_marks_to_read}, block_size{block_size}, storage{storage}, - use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions}, + : thread{thread}, pool{pool}, block_size_marks{block_size / storage.index_granularity}, + min_marks_to_read{(min_marks_to_read + block_size_marks - 1) / block_size_marks * block_size_marks}, + storage{storage}, use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions}, prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{settings.min_bytes_to_use_direct_io}, max_read_buffer_size{settings.max_read_buffer_size}, reuse_buffers(settings.merge_tree_uniform_read_reuse_buffers), virt_column_names{virt_column_names}, @@ -33,6 +34,7 @@ public: String getID() const override { std::stringstream res; + /// @todo print some meaningful information // res << "MergeTreeThread(columns"; // // for (const auto & column : columns) @@ -128,7 +130,7 @@ private: do { /// Прочитаем полный блок столбцов, нужных для вычисления выражения в PREWHERE. - size_t space_left = std::max(1LU, block_size / storage.index_granularity); + size_t space_left = std::max(1LU, block_size_marks); MarkRanges ranges_to_read; while (!task->mark_ranges.empty() && space_left) { @@ -257,7 +259,7 @@ private: } else { - size_t space_left = std::max(1LU, block_size / storage.index_granularity); + size_t space_left = std::max(1LU, block_size_marks); while (!task->mark_ranges.empty() && space_left) { @@ -310,8 +312,8 @@ private: } MergeTreeReadPoolPtr pool; + const std::size_t block_size_marks; const std::size_t min_marks_to_read; - const std::size_t block_size; MergeTreeData & storage; const bool use_uncompressed_cache; ExpressionActionsPtr prewhere_actions; From 9cc9cb241fe3548089d803afd46b3974bccb4bcb Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Wed, 2 Sep 2015 20:11:47 +0300 Subject: [PATCH 35/50] Merge --- dbms/include/DB/Storages/MergeTree/MergeTreeReader.h | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index 13903ace69e..697d1cdc192 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -42,10 +42,6 @@ public: const String & path, const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns, const MarkRanges & all_mark_ranges) { - /// @todo should avg_value_size_hints be cleared on this->data_part != data_part? supposedly so - if (this->data_part != data_part) - avg_value_size_hints.clear(); - if (!reuse_buffers) buffers.clear(); From f46326c4e8742b1375839eb5fd0f07946f767fc3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2015 20:47:29 +0300 Subject: [PATCH 36/50] dbms: StorageBuffer: better [#METR-17889]. --- dbms/src/Storages/StorageBuffer.cpp | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 269ab8a3a35..d95862ebbdb 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -275,7 +275,14 @@ void StorageBuffer::shutdown() if (flush_thread.joinable()) flush_thread.join(); - optimize(context.getSettings()); + try + { + optimize(context.getSettings()); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } From df90e447fbd919d69b24e5dab3b4272aa5c68624 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2015 22:39:19 +0300 Subject: [PATCH 37/50] dbms: increased limit for max_suspicious_broken_parts [#METR-17811]. --- dbms/include/DB/Storages/MergeTree/MergeTreeSettings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeSettings.h b/dbms/include/DB/Storages/MergeTree/MergeTreeSettings.h index eab98f49d5d..e3e14aa214d 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeSettings.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeSettings.h @@ -66,7 +66,7 @@ struct MergeTreeSettings size_t replicated_logs_to_keep = 100; /// Настройки минимального количества битых данных, при котором отказываться автоматически их удалять. - size_t max_suspicious_broken_parts = 5; + size_t max_suspicious_broken_parts = 10; /// Максимальное количество ошибок при загрузке кусков, при котором ReplicatedMergeTree соглашается запускаться. size_t replicated_max_unexpected_parts = 3; From 57c03c0dce3ea9c4c671a2a01544ac64e2d65943 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2015 22:58:29 +0300 Subject: [PATCH 38/50] dbms: whitespace [#METR-17552]. --- dbms/src/Storages/ITableDeclaration.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/ITableDeclaration.cpp b/dbms/src/Storages/ITableDeclaration.cpp index 83badf65ebe..7a99ebd7977 100644 --- a/dbms/src/Storages/ITableDeclaration.cpp +++ b/dbms/src/Storages/ITableDeclaration.cpp @@ -259,7 +259,7 @@ void ITableDeclaration::check(const Block & block, bool need_all) const const ColumnWithTypeAndName & column = block.getByPosition(i); if (names_in_block.count(column.name)) - throw Exception("Duplicate column " + column.name + " in block", + throw Exception("Duplicate column " + column.name + " in block", ErrorCodes::DUPLICATE_COLUMN); names_in_block.insert(column.name); From 0e26b649687ad382b33d96ab7b537db6c9a13959 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Sep 2015 23:35:38 +0300 Subject: [PATCH 39/50] dbms: allowed columns with duplicate names in GLOBAL subqueries [#METR-17552]. --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 37 +++++++++++++++++++ ..._subquery_columns_with_same_name.reference | 2 + ...global_subquery_columns_with_same_name.sql | 2 + 3 files changed, 41 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.reference create mode 100644 dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.sql diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index c02ee22684a..dc284fee321 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -886,8 +886,45 @@ static SharedPtr interpretSubquery( select_query->children.emplace_back(select_query->table); } else + { query = subquery->children.at(0); + /** В подзапросе могут быть указаны столбцы с одинаковыми именами. Например, SELECT x, x FROM t + * Это плохо, потому что результат такого запроса нельзя сохранить в таблицу, потому что в таблице не может быть одноимённых столбцов. + * Сохранение в таблицу требуется для GLOBAL-подзапросов. + * + * Чтобы избежать такой ситуации, будем переименовывать одинаковые столбцы. + */ + + std::set all_column_names; + std::set assigned_column_names; + + if (ASTSelectQuery * select = typeid_cast(query.get())) + { + for (auto & expr : select->select_expression_list->children) + all_column_names.insert(expr->getAliasOrColumnName()); + + for (auto & expr : select->select_expression_list->children) + { + auto name = expr->getAliasOrColumnName(); + + if (!assigned_column_names.insert(name).second) + { + size_t i = 1; + while (all_column_names.end() != all_column_names.find(name + "_" + toString(i))) + ++i; + + name = name + "_" + toString(i); + expr = expr->clone(); /// Отменяет склейку одинаковых выражений в дереве. + expr->setAlias(name); + + all_column_names.insert(name); + assigned_column_names.insert(name); + } + } + } + } + if (required_columns.empty()) return new InterpreterSelectQuery(query, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1); else diff --git a/dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.reference b/dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.reference new file mode 100644 index 00000000000..7589f01b456 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.reference @@ -0,0 +1,2 @@ +42 1 +1 diff --git a/dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.sql b/dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.sql new file mode 100644 index 00000000000..327fee9a18d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00217_global_subquery_columns_with_same_name.sql @@ -0,0 +1,2 @@ +SELECT k, a FROM (SELECT 42 AS k FROM remote('127.0.0.1', system.one)) GLOBAL ALL FULL OUTER JOIN (SELECT 42 AS k, 1 AS a, a) USING k; +SELECT 1 FROM remote('127.0.0.1', system.one) WHERE (1, 1) GLOBAL IN (SELECT 1 AS a, a); From 5ca53c8b10ba610f0ca08a180c85c896bef5c794 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 3 Sep 2015 12:17:25 +0300 Subject: [PATCH 40/50] Merge --- .../DB/Storages/MergeTree/MergeTreeReadPool.h | 15 +++++---------- .../DB/Storages/MergeTree/MergeTreeReader.h | 4 +--- 2 files changed, 6 insertions(+), 13 deletions(-) diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h index b4b35512397..edcadfa8387 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReadPool.h @@ -86,7 +86,6 @@ public: { const auto marks_to_get_from_range = marks_in_part; - /// @todo fix double reverse /// Восстановим порядок отрезков. std::reverse(thread_task.ranges.begin(), thread_task.ranges.end()); @@ -105,7 +104,6 @@ public: /// Цикл по отрезкам куска. while (need_marks > 0 && !thread_task.ranges.empty()) { - /// @todo fix double reverse auto & range = thread_task.ranges.back(); const std::size_t marks_in_range = range.end - range.begin; @@ -143,9 +141,7 @@ public: /// Посчитаем засечки для каждого куска. size_t sum_marks = 0; - /// Пусть отрезки будут перечислены справа налево, чтобы можно было выбрасывать самый левый отрезок с помощью pop_back(). -// std::reverse(std::begin(part.ranges), std::end(part.ranges)); - + /// Отрезки уже перечислены справа налево, reverse в MergeTreeDataSelectExecutor. for (const auto & range : part.ranges) sum_marks += range.end - range.begin; @@ -256,10 +252,7 @@ public: /// Возьмем весь кусок, если он достаточно мал. if (marks_in_part <= need_marks) { - /// @todo fix double reverse - /// Восстановим порядок отрезков. - std::reverse(part.ranges.begin(), part.ranges.end()); - + /// Оставим отрезки перечисленными справа налево для удобства. ranges_to_get_from_part = part.ranges; marks_in_ranges = marks_in_part; @@ -272,7 +265,6 @@ public: /// Цикл по отрезкам куска. while (need_marks > 0) { - /// @todo fix double reverse if (part.ranges.empty()) throw Exception("Unexpected end of ranges while spreading marks among threads", ErrorCodes::LOGICAL_ERROR); @@ -288,6 +280,9 @@ public: if (range.begin == range.end) part.ranges.pop_back(); } + + /// Вновь перечислим отрезки справа налево, чтобы .getTask() мог забирать их с помощью .pop_back(). + std::reverse(std::begin(ranges_to_get_from_part), std::end(ranges_to_get_from_part)); } threads_tasks[i].parts_and_ranges.push_back({ part_idx, ranges_to_get_from_part }); diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index 697d1cdc192..6d26f56e7b7 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -165,9 +165,6 @@ private: std::string path_prefix; size_t max_mark_range; - /// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины. - double avg_value_size_hint = 0; - 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, Memory & memory) @@ -310,6 +307,7 @@ private: * (ReadBufferAIO passes pointer to buffer to a syscall and waits for it's completion in destructor, thus there is * a chance that system will write to memory after it has been freed */ std::vector> buffers; + /// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины. std::map avg_value_size_hints; String path; MergeTreeData::DataPartPtr data_part; From 11857853a4e7d917e0bcafed117aab22637e1908 Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 3 Sep 2015 12:20:04 +0300 Subject: [PATCH 41/50] Merge --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 88c7c2958d9..474b395b9aa 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -371,7 +371,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongThreads( BlockInputStreams res; - /// @todo remove old code if (sum_marks > 0 && settings.merge_tree_uniform_read_distribution == 1) { MergeTreeReadPoolPtr pool = std::make_shared( From e8111d7e07692c3aa74eee48a4bfbda09d64c79d Mon Sep 17 00:00:00 2001 From: Andrey Mironov Date: Thu, 3 Sep 2015 15:07:46 +0300 Subject: [PATCH 42/50] Merge --- .../DB/IO/CachedCompressedReadBuffer.h | 28 ++---------- .../DB/IO/CompressedReadBufferFromFile.h | 5 +-- dbms/include/DB/Interpreters/Settings.h | 2 - .../DB/Storages/MergeTree/MergeTreeReader.h | 43 +++---------------- .../MergeTreeThreadBlockInputStream.h | 15 ++++--- dbms/src/IO/ReadBufferAIO.cpp | 5 +-- 6 files changed, 23 insertions(+), 75 deletions(-) diff --git a/dbms/include/DB/IO/CachedCompressedReadBuffer.h b/dbms/include/DB/IO/CachedCompressedReadBuffer.h index a2067f0e4e3..510e93b116a 100644 --- a/dbms/include/DB/IO/CachedCompressedReadBuffer.h +++ b/dbms/include/DB/IO/CachedCompressedReadBuffer.h @@ -19,7 +19,6 @@ class CachedCompressedReadBuffer : public CompressedReadBufferBase, public ReadB private: const std::string path; UncompressedCache * cache; - Memory * memory; size_t buf_size; size_t estimated_size; size_t aio_threshold; @@ -35,26 +34,7 @@ private: { if (!file_in) { - if (memory) - { - auto & memory = *this->memory; - - const auto resize = [&memory] (const std::size_t size) { - const auto growth_factor = 1.6f; /// close to golden ratio - if (memory.m_capacity == 0) - memory.resize(size); - else if (memory.m_capacity < size) - memory.resize(growth_factor * size); - }; - - if (aio_threshold == 0 || estimated_size < aio_threshold) - resize(buf_size); - else - resize(2 * Memory::align(buf_size + DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE)); - } - - file_in = createReadBufferFromFileBase( - path, estimated_size, aio_threshold, buf_size, -1, memory ? &(*memory)[0] : nullptr); + file_in = createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size); compressed_in = &*file_in; } } @@ -104,9 +84,9 @@ private: public: CachedCompressedReadBuffer( const std::string & path_, UncompressedCache * cache_, size_t estimated_size_, size_t aio_threshold_, - size_t buf_size_ = DBMS_DEFAULT_BUFFER_SIZE, Memory * memory = nullptr) - : ReadBuffer(nullptr, 0), path(path_), cache(cache_), memory{memory}, buf_size(buf_size_), - estimated_size(estimated_size_), aio_threshold(aio_threshold_), file_pos(0) + 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) { } diff --git a/dbms/include/DB/IO/CompressedReadBufferFromFile.h b/dbms/include/DB/IO/CompressedReadBufferFromFile.h index 3f7a1770a57..a8aa293e679 100644 --- a/dbms/include/DB/IO/CompressedReadBufferFromFile.h +++ b/dbms/include/DB/IO/CompressedReadBufferFromFile.h @@ -43,10 +43,9 @@ private: public: CompressedReadBufferFromFile( - const std::string & path, size_t estimated_size, size_t aio_threshold, - size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, char * existing_memory = nullptr) + const std::string & path, size_t estimated_size, size_t aio_threshold, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE) : BufferWithOwnMemory(0), - p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size, -1, existing_memory)), + p_file_in(createReadBufferFromFileBase(path, estimated_size, aio_threshold, buf_size)), file_in(*p_file_in) { compressed_in = &file_in; diff --git a/dbms/include/DB/Interpreters/Settings.h b/dbms/include/DB/Interpreters/Settings.h index 2251b020dac..9f3c5a384eb 100644 --- a/dbms/include/DB/Interpreters/Settings.h +++ b/dbms/include/DB/Interpreters/Settings.h @@ -121,8 +121,6 @@ struct Settings \ /** Распределять чтение из MergeTree по потокам равномерно, обеспечивая стабильное среднее время исполнения каждого потока в пределах одного чтения. */ \ M(SettingBool, merge_tree_uniform_read_distribution, false) \ - /** Переиспользовать буферы в MergeTreeReader::reconf() после выполнения подзадачи на чтение. */ \ - M(SettingBool, merge_tree_uniform_read_reuse_buffers, false) \ \ /** Минимальная длина выражения expr = x1 OR ... expr = xN для оптимизации */ \ M(SettingUInt64, optimize_min_equality_disjunction_chain_length, 3) \ diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h index 6d26f56e7b7..317fe4f9c65 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeReader.h @@ -31,9 +31,9 @@ public: const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns_, UncompressedCache * uncompressed_cache_, MarkCache * mark_cache_, MergeTreeData & storage_, const MarkRanges & all_mark_ranges, - size_t aio_threshold_, size_t max_read_buffer_size_, bool reuse_buffers = false) + size_t aio_threshold_, size_t max_read_buffer_size_) : uncompressed_cache(uncompressed_cache_), mark_cache(mark_cache_), storage(storage_), - aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_), reuse_buffers{reuse_buffers} + aio_threshold(aio_threshold_), max_read_buffer_size(max_read_buffer_size_) { reconf(path_, data_part, columns_, all_mark_ranges); } @@ -42,9 +42,6 @@ public: const String & path, const MergeTreeData::DataPartPtr & data_part, const NamesAndTypesList & columns, const MarkRanges & all_mark_ranges) { - if (!reuse_buffers) - buffers.clear(); - this->path = path; this->data_part = data_part; this->part_name = data_part->name; @@ -52,8 +49,6 @@ public: this->all_mark_ranges = all_mark_ranges; this->streams.clear(); - /// @todo sort buffers using capacity, find best match for Stream. - try { if (!Poco::File(path).exists()) @@ -167,7 +162,7 @@ private: 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, Memory & memory) + const MarkRanges & all_mark_ranges, size_t aio_threshold, size_t max_read_buffer_size) : path_prefix(path_prefix_) { loadMarks(mark_cache); @@ -220,28 +215,13 @@ private: if (uncompressed_cache) { cached_buffer = std::make_unique( - path_prefix + ".bin", uncompressed_cache, estimated_size, aio_threshold, buffer_size, &memory); + path_prefix + ".bin", uncompressed_cache, estimated_size, aio_threshold, buffer_size); data_buffer = cached_buffer.get(); } else { - const auto resize = [&memory] (const std::size_t size) { - const auto growth_factor = 1.6f; /// close to golden ratio - if (memory.m_capacity == 0) - memory.resize(size); - else if (memory.m_capacity < size) - memory.resize(growth_factor * size); - }; - - if (aio_threshold == 0 || estimated_size < aio_threshold) - resize(buffer_size); - else - resize(2 * Memory::align(buffer_size + DEFAULT_AIO_FILE_BLOCK_SIZE, DEFAULT_AIO_FILE_BLOCK_SIZE)); - - /** @todo CompressedReadBufferFromFile creates buffer for decompressed blocks, consider providing another - * instance of Memory type for it */ non_cached_buffer = std::make_unique( - path_prefix + ".bin", estimated_size, aio_threshold, buffer_size, &memory[0]); + path_prefix + ".bin", estimated_size, aio_threshold, buffer_size); data_buffer = non_cached_buffer.get(); } } @@ -303,10 +283,6 @@ private: typedef std::map > FileStreams; - /** buffers shall be deleted after streams because some streams may use existing_memory even inside destructor - * (ReadBufferAIO passes pointer to buffer to a syscall and waits for it's completion in destructor, thus there is - * a chance that system will write to memory after it has been freed */ - std::vector> buffers; /// Используется в качестве подсказки, чтобы уменьшить количество реаллокаций при создании столбца переменной длины. std::map avg_value_size_hints; String path; @@ -324,7 +300,6 @@ private: MarkRanges all_mark_ranges; size_t aio_threshold; size_t max_read_buffer_size; - bool reuse_buffers; void addStream(const String & name, const IDataType & type, const MarkRanges & all_mark_ranges, size_t level = 0) { @@ -336,10 +311,6 @@ private: if (!Poco::File(path + escaped_column_name + ".bin").exists()) return; - const auto buffer_idx = streams.size(); - if (buffer_idx == buffers.size()) - buffers.push_back(std::make_unique(0, DEFAULT_AIO_FILE_BLOCK_SIZE)); - /// Для массивов используются отдельные потоки для размеров. if (const DataTypeArray * type_arr = typeid_cast(&type)) { @@ -351,14 +322,14 @@ private: if (!streams.count(size_name)) streams.emplace(size_name, std::make_unique( path + escaped_size_name, uncompressed_cache, mark_cache, - all_mark_ranges, aio_threshold, max_read_buffer_size, *buffers[buffer_idx])); + all_mark_ranges, aio_threshold, max_read_buffer_size)); addStream(name, *type_arr->getNestedType(), all_mark_ranges, level + 1); } else streams.emplace(name, std::make_unique( path + escaped_column_name, uncompressed_cache, mark_cache, - all_mark_ranges, aio_threshold, max_read_buffer_size, *buffers[buffer_idx])); + all_mark_ranges, aio_threshold, max_read_buffer_size)); } diff --git a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h index e79d36fe83f..11d7860dfe8 100644 --- a/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h +++ b/dbms/include/DB/Storages/MergeTree/MergeTreeThreadBlockInputStream.h @@ -21,11 +21,15 @@ public: MergeTreeData & storage, const bool use_uncompressed_cache, const ExpressionActionsPtr & prewhere_actions, const String & prewhere_column, const Settings & settings, const Names & virt_column_names) : thread{thread}, pool{pool}, block_size_marks{block_size / storage.index_granularity}, - min_marks_to_read{(min_marks_to_read + block_size_marks - 1) / block_size_marks * block_size_marks}, + /// round min_marks_to_read up to nearest multiple of block_size expressed in marks + min_marks_to_read{block_size + ? (min_marks_to_read * storage.index_granularity + block_size - 1) + / block_size * block_size / storage.index_granularity + : min_marks_to_read + }, storage{storage}, use_uncompressed_cache{use_uncompressed_cache}, prewhere_actions{prewhere_actions}, prewhere_column{prewhere_column}, min_bytes_to_use_direct_io{settings.min_bytes_to_use_direct_io}, - max_read_buffer_size{settings.max_read_buffer_size}, - reuse_buffers(settings.merge_tree_uniform_read_reuse_buffers), virt_column_names{virt_column_names}, + max_read_buffer_size{settings.max_read_buffer_size}, virt_column_names{virt_column_names}, log{&Logger::get("MergeTreeThreadBlockInputStream")} {} @@ -103,13 +107,13 @@ private: reader = std::make_unique( path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), - storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size, reuse_buffers); + storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size); if (prewhere_actions) pre_reader = std::make_unique( path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), storage, task->mark_ranges, min_bytes_to_use_direct_io, - max_read_buffer_size, reuse_buffers); + max_read_buffer_size); } else { @@ -320,7 +324,6 @@ private: const String prewhere_column; const std::size_t min_bytes_to_use_direct_io; const std::size_t max_read_buffer_size; - const bool reuse_buffers; const Names virt_column_names; Logger * log; diff --git a/dbms/src/IO/ReadBufferAIO.cpp b/dbms/src/IO/ReadBufferAIO.cpp index ed48f362e3b..99f9e70f429 100644 --- a/dbms/src/IO/ReadBufferAIO.cpp +++ b/dbms/src/IO/ReadBufferAIO.cpp @@ -13,10 +13,7 @@ namespace DB /// не влезают в основной буфер. ReadBufferAIO::ReadBufferAIO(const std::string & filename_, size_t buffer_size_, int flags_, char * existing_memory_) : ReadBufferFromFileBase(buffer_size_ + DEFAULT_AIO_FILE_BLOCK_SIZE, existing_memory_, DEFAULT_AIO_FILE_BLOCK_SIZE), - fill_buffer(BufferWithOwnMemory(internalBuffer().size(), - existing_memory_ ? existing_memory_ + Memory::align(internalBuffer().size(), DEFAULT_AIO_FILE_BLOCK_SIZE) - : nullptr, - DEFAULT_AIO_FILE_BLOCK_SIZE)), + fill_buffer(BufferWithOwnMemory(internalBuffer().size(), nullptr, DEFAULT_AIO_FILE_BLOCK_SIZE)), filename(filename_) { ProfileEvents::increment(ProfileEvents::FileOpen); From 1491260e3be7277674da47199db6caa4c7212652 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 3 Sep 2015 15:09:47 +0300 Subject: [PATCH 43/50] dbms: Server: Fixed behavior of LIKE. Now % correctly matches newlines. [#METR-17588] --- dbms/include/DB/Functions/FunctionsStringSearch.h | 2 +- .../0_stateless/00218_like_regexp_newline.reference | 11 +++++++++++ .../queries/0_stateless/00218_like_regexp_newline.sql | 11 +++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/00218_like_regexp_newline.reference create mode 100644 dbms/tests/queries/0_stateless/00218_like_regexp_newline.sql diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index f1a5a3a45a1..8bb893eaa55 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -180,7 +180,7 @@ inline String likePatternToRegexp(const String & pattern) break; case '%': if (pos + 1 != end) - res += ".*"; + res += "(?:.|\n)*"; else return res; break; diff --git a/dbms/tests/queries/0_stateless/00218_like_regexp_newline.reference b/dbms/tests/queries/0_stateless/00218_like_regexp_newline.reference new file mode 100644 index 00000000000..3716d57d54e --- /dev/null +++ b/dbms/tests/queries/0_stateless/00218_like_regexp_newline.reference @@ -0,0 +1,11 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00218_like_regexp_newline.sql b/dbms/tests/queries/0_stateless/00218_like_regexp_newline.sql new file mode 100644 index 00000000000..824bbb66ad9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00218_like_regexp_newline.sql @@ -0,0 +1,11 @@ +SELECT 'abcdef' LIKE '%abc%def%'; +SELECT 'abctdef' LIKE '%abc%def%'; +SELECT 'abc\ndef' LIKE '%abc%def%'; +SELECT 'abc\ntdef' LIKE '%abc%def%'; +SELECT 'abct\ndef' LIKE '%abc%def%'; +SELECT 'abc\n\ndef' LIKE '%abc%def%'; +SELECT 'abc\n\ntdef' LIKE '%abc%def%'; +SELECT 'abc\nt\ndef' LIKE '%abc%def%'; +SELECT 'abct\n\ndef' LIKE '%abc%def%'; +SELECT 'ab\ndef' LIKE '%abc%def%'; +SELECT 'abc\nef' LIKE '%abc%def%'; From 1dc1cbc0219d4178a104c465bb9c0441f2117a21 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 3 Sep 2015 17:21:55 +0300 Subject: [PATCH 44/50] Merge --- dbms/scripts/gen-bias-data.py | 108 ++++++++++++---------- dbms/scripts/linear-counting-threshold.py | 107 +++++++++++---------- 2 files changed, 116 insertions(+), 99 deletions(-) diff --git a/dbms/scripts/gen-bias-data.py b/dbms/scripts/gen-bias-data.py index 2d034b0976c..f016f8fc6e3 100755 --- a/dbms/scripts/gen-bias-data.py +++ b/dbms/scripts/gen-bias-data.py @@ -9,6 +9,7 @@ import subprocess import bisect from copy import deepcopy +# Псевдослучайный генератор уникальных чисел. # http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/ class UniqueRandomGenerator: prime = 4294967291 @@ -32,8 +33,9 @@ class UniqueRandomGenerator: else: return self.prime - residue -def generate_data_source(host, port, http_port, begin, end, count): - chunk_size = round((end - begin) / float(count)) +# Создать таблицу содержащую уникальные значения. +def generate_data_source(host, port, http_port, min_cardinality, max_cardinality, count): + chunk_size = round((max_cardinality - min_cardinality) / float(count)) used_values = 0 cur_count = 0 @@ -44,30 +46,36 @@ def generate_data_source(host, port, http_port, begin, end, count): n2 = random.randrange(0, sup) urng = UniqueRandomGenerator(n1, n2) + is_first = True + with tempfile.TemporaryDirectory() as tmp_dir: filename = tmp_dir + '/table.txt' - file_handle = open(filename, 'w+b') + with open(filename, 'w+b') as file_handle: + while cur_count < count: - while cur_count < count: - next_size += chunk_size + if is_first == True: + is_first = False + if min_cardinality != 0: + next_size = min_cardinality + 1 + else: + next_size = chunk_size + else: + next_size += chunk_size - while used_values < next_size: - h = urng.next() - used_values = used_values + 1 - outstr = str(h) + "\t" + str(cur_count) + "\n"; - file_handle.write(bytes(outstr, 'UTF-8')); + while used_values < next_size: + h = urng.next() + used_values = used_values + 1 + out = str(h) + "\t" + str(cur_count) + "\n"; + file_handle.write(bytes(out, 'UTF-8')); + cur_count = cur_count + 1 - cur_count = cur_count + 1 - - file_handle.close() - - query = 'DROP TABLE IF EXISTS data_source' + query = "DROP TABLE IF EXISTS data_source" subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) - query = 'CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog' + query = "CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog" subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE) - subprocess.check_output(("POST", "http://localhost:{0}/?query=INSERT INTO data_source FORMAT TabSeparated".format(http_port)), stdin=cat.stdout) + subprocess.check_output(("POST", "http://{0}:{1}/?query=INSERT INTO data_source FORMAT TabSeparated".format(host, http_port)), stdin=cat.stdout) cat.wait() def perform_query(host, port): @@ -76,27 +84,27 @@ def perform_query(host, port): query += "FROM data_source GROUP BY KeyID" return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query]) -def parse_clickhouse_response(output): +def parse_clickhouse_response(response): parsed = [] - lines = output.decode().split("\n") + lines = response.decode().split("\n") for cur_line in lines: rows = cur_line.split("\t") if len(rows) == 2: parsed.append([float(rows[0]), float(rows[1])]) return parsed -def accumulate_data(stats, data): - if not stats: - stats = deepcopy(data) +def accumulate_data(accumulated_data, data): + if not accumulated_data: + accumulated_data = deepcopy(data) else: - for row1, row2 in zip(stats, data): + for row1, row2 in zip(accumulated_data, data): row1[1] += row2[1]; - return stats + return accumulated_data -def generate_raw_result(stats, count): +def generate_raw_result(accumulated_data, count): expected_tab = [] bias_tab = [] - for row in stats: + for row in accumulated_data: exact = row[0] expected = row[1] / count bias = expected - exact @@ -110,7 +118,7 @@ def generate_sample(raw_estimates, biases, n_samples): min_card = raw_estimates[0] max_card = raw_estimates[len(raw_estimates) - 1] - step = (max_card - min_card) / n_samples + step = (max_card - min_card) / (n_samples - 1) for i in range(0, n_samples + 1): x = min_card + i * step @@ -197,56 +205,58 @@ def generate_sample(raw_estimates, biases, n_samples): return final_result -def dump_arrays(stats): +def dump_arrays(data): - print("Size of each array: {0}\n".format(len(stats))) + print("Size of each array: {0}\n".format(len(data))) is_first = True sep = '' - print("// For UniqCombinedBiasData::getRawEstimates():") + print("raw_estimates = ") print("{") - for row in stats: + for row in data: print("\t{0}{1}".format(sep, row[0])) if is_first == True: is_first = False sep = "," - print("}") + print("};") is_first = True sep = "" - print("\n// For UniqCombinedBiasData::getBiases():") + print("\nbiases = ") print("{") - for row in stats: + for row in data: print("\t{0}{1}".format(sep, row[1])) if is_first == True: is_first = False sep = "," - print("}") + print("};") def start(): - parser = argparse.ArgumentParser(description = "Generate bias correction tables.") - parser.add_argument("-x", "--host", default="127.0.0.1", help="clickhouse host name"); - parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port"); - parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port"); + parser = argparse.ArgumentParser(description = "Generate bias correction tables for HyperLogLog-based functions.") + parser.add_argument("-x", "--host", default="127.0.0.1", help="ClickHouse server host name"); + parser.add_argument("-p", "--port", type=int, default=9000, help="ClickHouse server TCP port"); + parser.add_argument("-t", "--http_port", type=int, default=8123, help="ClickHouse server HTTP port"); parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations"); - parser.add_argument("-s", "--generated", type=int, default=700000, help="number of generated values"); - parser.add_argument("-g", "--samples", type=int, default=200, help="number of sampled values"); + parser.add_argument("-m", "--min_cardinality", type=int, default=16384, help="minimal cardinality"); + parser.add_argument("-M", "--max_cardinality", type=int, default=655360, help="maximal cardinality"); + parser.add_argument("-s", "--samples", type=int, default=200, help="number of sampled values"); args = parser.parse_args() - stats = [] + accumulated_data = [] for i in range(0, args.iterations): print(i + 1) sys.stdout.flush() - generate_data_source(args.host, str(args.port), str(args.http_port), 0, args.generated, 1000) - output = perform_query(args.host, str(args.port)) - data = parse_clickhouse_response(output) - stats = accumulate_data(stats, data) - result = generate_raw_result(stats, args.iterations) - sample = generate_sample(result[0], result[1], args.samples) - dump_arrays(sample) + generate_data_source(args.host, str(args.port), str(args.http_port), args.min_cardinality, args.max_cardinality, 1000) + response = perform_query(args.host, str(args.port)) + data = parse_clickhouse_response(response) + accumulated_data = accumulate_data(accumulated_data, data) + + result = generate_raw_result(accumulated_data, args.iterations) + sampled_data = generate_sample(result[0], result[1], args.samples) + dump_arrays(sampled_data) if __name__ == "__main__": start() diff --git a/dbms/scripts/linear-counting-threshold.py b/dbms/scripts/linear-counting-threshold.py index eac57d50775..c327daf6cd4 100755 --- a/dbms/scripts/linear-counting-threshold.py +++ b/dbms/scripts/linear-counting-threshold.py @@ -9,6 +9,7 @@ import subprocess import bisect from copy import deepcopy +# Псевдослучайный генератор уникальных чисел. # http://preshing.com/20121224/how-to-generate-a-sequence-of-unique-random-integers/ class UniqueRandomGenerator: prime = 4294967291 @@ -32,8 +33,9 @@ class UniqueRandomGenerator: else: return self.prime - residue -def generate_data_source(host, port, http_port, begin, end, count): - chunk_size = round((end - begin) / float(count)) +# Создать таблицу содержащую уникальные значения. +def generate_data_source(host, port, http_port, min_cardinality, max_cardinality, count): + chunk_size = round((max_cardinality - (min_cardinality + 1)) / float(count)) used_values = 0 cur_count = 0 @@ -44,30 +46,36 @@ def generate_data_source(host, port, http_port, begin, end, count): n2 = random.randrange(0, sup) urng = UniqueRandomGenerator(n1, n2) + is_first = True + with tempfile.TemporaryDirectory() as tmp_dir: filename = tmp_dir + '/table.txt' - file_handle = open(filename, 'w+b') + with open(filename, 'w+b') as file_handle: + while cur_count < count: - while cur_count < count: - next_size += chunk_size + if is_first == True: + is_first = False + if min_cardinality != 0: + next_size = min_cardinality + 1 + else: + next_size = chunk_size + else: + next_size += chunk_size - while used_values < next_size: - h = urng.next() - used_values = used_values + 1 - outstr = str(h) + "\t" + str(cur_count) + "\n"; - file_handle.write(bytes(outstr, 'UTF-8')); + while used_values < next_size: + h = urng.next() + used_values = used_values + 1 + out = str(h) + "\t" + str(cur_count) + "\n"; + file_handle.write(bytes(out, 'UTF-8')); + cur_count = cur_count + 1 - cur_count = cur_count + 1 - - file_handle.close() - - query = 'DROP TABLE IF EXISTS data_source' + query = "DROP TABLE IF EXISTS data_source" subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) - query = 'CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog' + query = "CREATE TABLE data_source(UserID UInt64, KeyID UInt64) ENGINE=TinyLog" subprocess.check_output(["clickhouse-client", "--host", host, "--port", str(port), "--query", query]) cat = subprocess.Popen(("cat", filename), stdout=subprocess.PIPE) - subprocess.check_output(("POST", "http://localhost:{0}/?query=INSERT INTO data_source FORMAT TabSeparated".format(http_port)), stdin=cat.stdout) + subprocess.check_output(("POST", "http://{0}:{1}/?query=INSERT INTO data_source FORMAT TabSeparated".format(host, http_port)), stdin=cat.stdout) cat.wait() def perform_query(host, port): @@ -78,66 +86,65 @@ def perform_query(host, port): query += "FROM data_source GROUP BY KeyID" return subprocess.check_output(["clickhouse-client", "--host", host, "--port", port, "--query", query]) -def parse_clickhouse_response(output): +def parse_clickhouse_response(response): parsed = [] - lines = output.decode().split("\n") + lines = response.decode().split("\n") for cur_line in lines: rows = cur_line.split("\t") if len(rows) == 4: parsed.append([float(rows[0]), float(rows[1]), float(rows[2]), float(rows[3])]) return parsed -def accumulate_data(stats, data): - if not stats: - stats = deepcopy(data) +def accumulate_data(accumulated_data, data): + if not accumulated_data: + accumulated_data = deepcopy(data) else: - for row1, row2 in zip(stats, data): + for row1, row2 in zip(accumulated_data, data): row1[1] += row2[1]; row1[2] += row2[2]; row1[3] += row2[3]; - return stats + return accumulated_data -def dump_graphs(stats, count): - fh1 = open("raw_graph.txt", "w+b") - fh2 = open("linear_counting_graph.txt", "w+b") - fh3 = open("bias_corrected_graph.txt", "w+b") +def dump_graphs(data, count): + with open("raw_graph.txt", "w+b") as fh1, open("linear_counting_graph.txt", "w+b") as fh2, open("bias_corrected_graph.txt", "w+b") as fh3: + expected_tab = [] + bias_tab = [] + for row in data: + exact = row[0] + raw = row[1] / count; + linear_counting = row[2] / count; + bias_corrected = row[3] / count; - expected_tab = [] - bias_tab = [] - for row in stats: - exact = row[0] - raw = row[1] / count; - linear_counting = row[2] / count; - bias_corrected = row[3] / count; + outstr = "{0}\t{1}\n".format(exact, abs(raw - exact) / exact) + fh1.write(bytes(outstr, 'UTF-8')) - outstr = "{0}\t{1}\n".format(exact, abs(raw - exact) / exact) - fh1.write(bytes(outstr, 'UTF-8')) + outstr = "{0}\t{1}\n".format(exact, abs(linear_counting - exact) / exact) + fh2.write(bytes(outstr, 'UTF-8')) - outstr = "{0}\t{1}\n".format(exact, abs(linear_counting - exact) / exact) - fh2.write(bytes(outstr, 'UTF-8')) - - outstr = "{0}\t{1}\n".format(exact, abs(bias_corrected - exact) / exact) - fh3.write(bytes(outstr, 'UTF-8')) + outstr = "{0}\t{1}\n".format(exact, abs(bias_corrected - exact) / exact) + fh3.write(bytes(outstr, 'UTF-8')) def start(): - parser = argparse.ArgumentParser(description = "Generate bias correction tables.") + parser = argparse.ArgumentParser(description = "Generate graphs that help to determine the linear counting threshold.") parser.add_argument("-x", "--host", default="127.0.0.1", help="clickhouse host name"); parser.add_argument("-p", "--port", type=int, default=9000, help="clickhouse client TCP port"); parser.add_argument("-t", "--http_port", type=int, default=8123, help="clickhouse HTTP port"); parser.add_argument("-i", "--iterations", type=int, default=5000, help="number of iterations"); - parser.add_argument("-s", "--generated", type=int, default=700000, help="number of generated values"); + parser.add_argument("-m", "--min_cardinality", type=int, default=16384, help="minimal cardinality"); + parser.add_argument("-M", "--max_cardinality", type=int, default=655360, help="maximal cardinality"); args = parser.parse_args() - stats = [] + accumulated_data = [] for i in range(0, args.iterations): print(i + 1) sys.stdout.flush() - generate_data_source(args.host, str(args.port), str(args.http_port), 0, args.generated, 1000) - output = perform_query(args.host, str(args.port)) - data = parse_clickhouse_response(output) - stats = accumulate_data(stats, data) - dump_graphs(stats, args.iterations) + generate_data_source(args.host, str(args.port), str(args.http_port), args.min_cardinality, args.max_cardinality, 1000) + response = perform_query(args.host, str(args.port)) + data = parse_clickhouse_response(response) + accumulated_data = accumulate_data(accumulated_data, data) + + dump_graphs(accumulated_data, args.iterations) if __name__ == "__main__": start() From d18544c981864c17382df6e11d6b9f384a072a15 Mon Sep 17 00:00:00 2001 From: Alexey Arno Date: Thu, 3 Sep 2015 18:35:34 +0300 Subject: [PATCH 45/50] dbms: Server: Improved implementation. [#METR-17588] --- dbms/include/DB/Functions/FunctionsStringSearch.h | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index 8bb893eaa55..914f5a82627 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -180,7 +180,7 @@ inline String likePatternToRegexp(const String & pattern) break; case '%': if (pos + 1 != end) - res += "(?:.|\n)*"; + res += ".*"; else return res; break; @@ -324,7 +324,12 @@ namespace Regexps it = known_regexps.emplace(pattern, std::make_unique()).first; return it->second->get([&pattern] { - return new Regexp{createRegexp(pattern, no_capture ? OptimizedRegularExpression::RE_NO_CAPTURE : 0)}; + int flags = 0; + if (like) + flags |= OptimizedRegularExpression::RE_DOT_NL; + if (no_capture) + flags |= OptimizedRegularExpression::RE_NO_CAPTURE; + return new Regexp{createRegexp(pattern, flags)}; }); } } From 69be9e7dd00f68fa2b3b5734cab5f7b6dec13b9b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Sep 2015 21:48:41 +0300 Subject: [PATCH 46/50] dbms: increased default limit on query size [#METRIKASUPP-5557]. --- dbms/include/DB/Core/Defines.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/include/DB/Core/Defines.h b/dbms/include/DB/Core/Defines.h index e25fb73fc80..dd559f80bbc 100644 --- a/dbms/include/DB/Core/Defines.h +++ b/dbms/include/DB/Core/Defines.h @@ -45,7 +45,7 @@ */ #define DEFAULT_MERGE_BLOCK_SIZE 8192 -#define DEFAULT_MAX_QUERY_SIZE 65536 +#define DEFAULT_MAX_QUERY_SIZE 262144 #define SHOW_CHARS_ON_SYNTAX_ERROR 160L #define DEFAULT_MAX_DISTRIBUTED_CONNECTIONS 1024 #define DEFAULT_INTERACTIVE_DELAY 100000 From bf6544693ecf33e72e55d3901d3e69f8dfbfb477 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Sep 2015 22:19:19 +0300 Subject: [PATCH 47/50] dbms: tiny modifications [#METR-17588]. --- dbms/include/DB/Functions/FunctionsStringSearch.h | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/include/DB/Functions/FunctionsStringSearch.h b/dbms/include/DB/Functions/FunctionsStringSearch.h index 914f5a82627..0d14e14bfc3 100644 --- a/dbms/include/DB/Functions/FunctionsStringSearch.h +++ b/dbms/include/DB/Functions/FunctionsStringSearch.h @@ -323,12 +323,12 @@ namespace Regexps if (known_regexps.end() == it) it = known_regexps.emplace(pattern, std::make_unique()).first; - return it->second->get([&pattern] { - int flags = 0; - if (like) - flags |= OptimizedRegularExpression::RE_DOT_NL; + return it->second->get([&pattern] + { + int flags = OptimizedRegularExpression::RE_DOT_NL; if (no_capture) flags |= OptimizedRegularExpression::RE_NO_CAPTURE; + return new Regexp{createRegexp(pattern, flags)}; }); } From 7dc7144011f2450d4c96713367ade0f88e42cea9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Sep 2015 23:48:02 +0300 Subject: [PATCH 48/50] dbms: fixed error with column order in RIGHT and FULL JOINs [#METR-17550]. --- .../DB/Interpreters/ExpressionAnalyzer.h | 4 +-- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 20 ++++++------ dbms/src/Interpreters/Join.cpp | 31 +++++++++++++------ 3 files changed, 34 insertions(+), 21 deletions(-) diff --git a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h index 36d7ea25e5e..d7111b1345f 100644 --- a/dbms/include/DB/Interpreters/ExpressionAnalyzer.h +++ b/dbms/include/DB/Interpreters/ExpressionAnalyzer.h @@ -163,8 +163,8 @@ private: * - в "левой" таблице, он будет доступен по имени expr(x), так как ещё не было выполнено действие Project. * Надо запомнить оба этих варианта. */ - NameSet join_key_names_left_set; - NameSet join_key_names_right_set; + Names join_key_names_left; + Names join_key_names_right; NamesAndTypesList columns_added_by_join; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index dc284fee321..2712ba8cc71 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -1780,8 +1780,6 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty if (!subquery_for_set.join) { - Names join_key_names_left(join_key_names_left_set.begin(), join_key_names_left_set.end()); - Names join_key_names_right(join_key_names_right_set.begin(), join_key_names_right_set.end()); JoinPtr join = new Join(join_key_names_left, join_key_names_right, settings.limits, ast_join.kind, ast_join.strictness); Names required_joined_columns(join_key_names_right.begin(), join_key_names_right.end()); @@ -2174,27 +2172,31 @@ void ExpressionAnalyzer::collectJoinedColumns(NameSet & joined_columns, NamesAnd auto & keys = typeid_cast(*node.using_expr_list); for (const auto & key : keys.children) { - if (!join_key_names_left_set.insert(key->getColumnName()).second) - throw Exception("Duplicate column in USING list", ErrorCodes::DUPLICATE_COLUMN); + if (join_key_names_left.end() == std::find(join_key_names_left.begin(), join_key_names_left.end(), key->getColumnName())) + join_key_names_left.push_back(key->getColumnName()); + else + throw Exception("Duplicate column " + key->getColumnName() + " in USING list", ErrorCodes::DUPLICATE_COLUMN); - if (!join_key_names_right_set.insert(key->getAliasOrColumnName()).second) - throw Exception("Duplicate column in USING list", ErrorCodes::DUPLICATE_COLUMN); + if (join_key_names_right.end() == std::find(join_key_names_right.begin(), join_key_names_right.end(), key->getAliasOrColumnName())) + join_key_names_right.push_back(key->getAliasOrColumnName()); + else + throw Exception("Duplicate column " + key->getAliasOrColumnName() + " in USING list", ErrorCodes::DUPLICATE_COLUMN); } } for (const auto i : ext::range(0, nested_result_sample.columns())) { const auto & col = nested_result_sample.getByPosition(i); - if (!join_key_names_right_set.count(col.name)) + if (join_key_names_right.end() == std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name)) { joined_columns.insert(col.name); joined_columns_name_type.emplace_back(col.name, col.type); } } -/* for (const auto & name : join_key_names_left_set) +/* for (const auto & name : join_key_names_left) std::cerr << "JOIN key (left): " << name << std::endl; - for (const auto & name : join_key_names_right_set) + for (const auto & name : join_key_names_right) std::cerr << "JOIN key (right): " << name << std::endl; std::cerr << std::endl; for (const auto & name : joined_columns) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index a0badc220e9..7b6a7831fe7 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -376,12 +376,18 @@ void Join::setSampleBlock(const Block & block) sample_block_with_columns_to_add = block; - /// Удаляем из sample_block_with_columns_to_add ключевые столбцы. - for (const auto & name : key_names_right) + /// Переносим из sample_block_with_columns_to_add ключевые столбцы в sample_block_with_keys, сохраняя порядок. + size_t pos = 0; + while (pos < sample_block_with_columns_to_add.columns()) { - size_t pos = sample_block_with_columns_to_add.getPositionByName(name); - sample_block_with_keys.insert(sample_block_with_columns_to_add.unsafeGetByPosition(pos)); - sample_block_with_columns_to_add.erase(pos); + const auto & name = sample_block_with_columns_to_add.unsafeGetByPosition(pos).name; + if (key_names_right.end() != std::find(key_names_right.begin(), key_names_right.end(), name)) + { + sample_block_with_keys.insert(sample_block_with_columns_to_add.unsafeGetByPosition(pos)); + sample_block_with_columns_to_add.erase(pos); + } + else + ++pos; } for (size_t i = 0, size = sample_block_with_columns_to_add.columns(); i < size; ++i) @@ -426,7 +432,9 @@ bool Join::insertFromBlock(const Block & block) if (getFullness(kind)) { - /// Переносим ключевые столбцы в начало блока. + /** Переносим ключевые столбцы в начало блока. + * Именно там их будет ожидать NonJoinedBlockInputStream. + */ size_t key_num = 0; for (const auto & name : key_names_right) { @@ -810,6 +818,8 @@ void Join::checkTypesOfKeys(const Block & block_left, const Block & block_right) void Join::joinBlock(Block & block) const { +// std::cerr << "joinBlock: " << block.dumpStructure() << "\n"; + Poco::ScopedReadRWLock lock(rwlock); checkTypesOfKeys(block, sample_block_with_keys); @@ -917,6 +927,8 @@ public: result_sample_block = left_sample_block; +// std::cerr << result_sample_block.dumpStructure() << "\n"; + /// Добавляем в блок новые столбцы. for (size_t i = 0; i < num_columns_right; ++i) { @@ -932,10 +944,11 @@ public: { const String & name = left_sample_block.getByPosition(i).name; - if (parent.key_names_left.end() == std::find(parent.key_names_left.begin(), parent.key_names_left.end(), name)) + auto found_key_column = std::find(parent.key_names_left.begin(), parent.key_names_left.end(), name); + if (parent.key_names_left.end() == found_key_column) column_numbers_left.push_back(i); else - column_numbers_keys_and_right.push_back(i); + column_numbers_keys_and_right.push_back(found_key_column - parent.key_names_left.begin()); } for (size_t i = 0; i < num_columns_right; ++i) @@ -1046,8 +1059,6 @@ private: for (; it != end; ++it) { -// std::cerr << it->second.getUsed() << "\n"; - if (it->second.getUsed()) continue; From e923dd2ff6ac26cb5afcbc21a5568c52342d0b0c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 3 Sep 2015 23:50:41 +0300 Subject: [PATCH 49/50] dbms: added test [#METR-17550]. --- .../00219_full_right_join_column_order.reference | 6 ++++++ .../0_stateless/00219_full_right_join_column_order.sql | 6 ++++++ 2 files changed, 12 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00219_full_right_join_column_order.reference create mode 100644 dbms/tests/queries/0_stateless/00219_full_right_join_column_order.sql diff --git a/dbms/tests/queries/0_stateless/00219_full_right_join_column_order.reference b/dbms/tests/queries/0_stateless/00219_full_right_join_column_order.reference new file mode 100644 index 00000000000..2f6676b0a17 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00219_full_right_join_column_order.reference @@ -0,0 +1,6 @@ +2 3000 +2 3000 +1 2000 +2 3000 +1 2000 +2 3000 diff --git a/dbms/tests/queries/0_stateless/00219_full_right_join_column_order.sql b/dbms/tests/queries/0_stateless/00219_full_right_join_column_order.sql new file mode 100644 index 00000000000..d033d197cf7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00219_full_right_join_column_order.sql @@ -0,0 +1,6 @@ +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING a, b; +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b) USING b, a; + +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) USING a, b; +SELECT a, b FROM (SELECT 1 AS a, 2000 AS b) ANY RIGHT JOIN (SELECT 2 AS a, 3000 AS b UNION ALL SELECT 1 AS a, 2000 AS b) USING b, a; + From 767a3c8090592a89c858abe78f3649c966505eec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 4 Sep 2015 22:40:55 +0300 Subject: [PATCH 50/50] dbms: fixed error with WITH TOTALS in subquery with distributed table with one shard and LIMIT [#METR-17561]. --- .../DB/DataStreams/LimitBlockInputStream.h | 10 ++++- .../src/DataStreams/LimitBlockInputStream.cpp | 15 +++++-- .../tests/filter_stream_hitlog.cpp | 2 +- dbms/src/DataStreams/tests/sorting_stream.cpp | 2 +- dbms/src/DataStreams/tests/union_stream2.cpp | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 45 +++++++++++++++---- dbms/src/Storages/tests/storage_log.cpp | 2 +- ...als_in_subquery_remote_and_limit.reference | 3 ++ ...th_totals_in_subquery_remote_and_limit.sql | 1 + 9 files changed, 65 insertions(+), 17 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.reference create mode 100644 dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.sql diff --git a/dbms/include/DB/DataStreams/LimitBlockInputStream.h b/dbms/include/DB/DataStreams/LimitBlockInputStream.h index b6cd2e086de..21d832eaf83 100644 --- a/dbms/include/DB/DataStreams/LimitBlockInputStream.h +++ b/dbms/include/DB/DataStreams/LimitBlockInputStream.h @@ -16,7 +16,12 @@ using Poco::SharedPtr; class LimitBlockInputStream : public IProfilingBlockInputStream { public: - LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_ = 0); + /** Если always_read_till_end = false (по-умолчанию), то после чтения достаточного количества данных, + * возвращает пустой блок, и это приводит к отмене выполнения запроса. + * Если always_read_till_end = true - читает все данные до конца, но игнорирует их. Это нужно в редких случаях: + * когда иначе, из-за отмены запроса, мы бы не получили данные для GROUP BY WITH TOTALS с удалённого сервера. + */ + LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_ = false); String getName() const override { return "Limit"; } @@ -33,7 +38,8 @@ protected: private: size_t limit; size_t offset; - size_t pos; + size_t pos = 0; + bool always_read_till_end; }; } diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 479bf555829..77214416702 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -8,8 +8,8 @@ namespace DB using Poco::SharedPtr; -LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_) - : limit(limit_), offset(offset_), pos(0) +LimitBlockInputStream::LimitBlockInputStream(BlockInputStreamPtr input_, size_t limit_, size_t offset_, bool always_read_till_end_) + : limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_) { children.push_back(input_); } @@ -23,7 +23,16 @@ Block LimitBlockInputStream::readImpl() /// pos - сколько строк было прочитано, включая последний прочитанный блок if (pos >= offset + limit) - return res; + { + if (!always_read_till_end) + return res; + else + { + while (children.back()->read()) + ; + return res; + } + } do { diff --git a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp index 05532efe961..1cf67c3cc03 100644 --- a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp +++ b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp @@ -132,7 +132,7 @@ int main(int argc, char ** argv) Poco::SharedPtr in = table->read(column_names, 0, context, Settings(), stage)[0]; in = new ExpressionBlockInputStream(in, expression); in = new FilterBlockInputStream(in, 4); - //in = new LimitBlockInputStream(in, 10); + //in = new LimitBlockInputStream(in, 10, 0); WriteBufferFromOStream ob(std::cout); RowOutputStreamPtr out_ = new TabSeparatedRowOutputStream(ob, expression->getSampleBlock()); diff --git a/dbms/src/DataStreams/tests/sorting_stream.cpp b/dbms/src/DataStreams/tests/sorting_stream.cpp index 15d24f7d6f8..6f881e237ae 100644 --- a/dbms/src/DataStreams/tests/sorting_stream.cpp +++ b/dbms/src/DataStreams/tests/sorting_stream.cpp @@ -152,7 +152,7 @@ int main(int argc, char ** argv) Poco::SharedPtr in = table->read(column_names, 0, Context{}, Settings(), stage, argc == 2 ? atoi(argv[1]) : 1048576)[0]; in = new PartialSortingBlockInputStream(in, sort_columns); in = new MergeSortingBlockInputStream(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, ""); - //in = new LimitBlockInputStream(in, 10); + //in = new LimitBlockInputStream(in, 10, 0); WriteBufferFromOStream ob(std::cout); RowOutputStreamPtr out_ = new TabSeparatedRowOutputStream(ob, sample); diff --git a/dbms/src/DataStreams/tests/union_stream2.cpp b/dbms/src/DataStreams/tests/union_stream2.cpp index c12b613dc93..f63b88193b4 100644 --- a/dbms/src/DataStreams/tests/union_stream2.cpp +++ b/dbms/src/DataStreams/tests/union_stream2.cpp @@ -45,7 +45,7 @@ int main(int argc, char ** argv) streams[i] = new DB::AsynchronousBlockInputStream(streams[i]); DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads); - stream = new DB::LimitBlockInputStream(stream, 10); + stream = new DB::LimitBlockInputStream(stream, 10, 0); DB::FormatFactory format_factory; DB::WriteBufferFromFileDescriptor wb(STDERR_FILENO); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index e5b73b21f66..7e844d1584e 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -98,7 +98,7 @@ void InterpreterSelectQuery::init(BlockInputStreamPtr input, const Names & requi void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_) { - if (query.table && typeid_cast(&*query.table)) + if (query.table && typeid_cast(query.table.get())) { if (table_column_names.empty()) { @@ -107,10 +107,10 @@ void InterpreterSelectQuery::basicInit(BlockInputStreamPtr input_) } else { - if (query.table && typeid_cast(&*query.table)) + if (query.table && typeid_cast(query.table.get())) { /// Получить табличную функцию - TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast(&*query.table)->name, context); + TableFunctionPtr table_function_ptr = context.getTableFunctionFactory().get(typeid_cast(query.table.get())->name, context); /// Выполнить ее и запомнить результат storage = table_function_ptr->execute(query.table, context); } @@ -329,7 +329,7 @@ BlockIO InterpreterSelectQuery::execute() executeUnion(); /// Ограничения на результат, квота на результат, а также колбек для прогресса. - if (IProfilingBlockInputStream * stream = dynamic_cast(&*streams[0])) + if (IProfilingBlockInputStream * stream = dynamic_cast(streams[0].get())) { /// Ограничения действуют только на конечный результат. if (to_stage == QueryProcessingStage::Complete) @@ -590,7 +590,7 @@ void InterpreterSelectQuery::executeSingleQuery() { transformStreams([&](auto & stream) { - if (IProfilingBlockInputStream * p_stream = dynamic_cast(&*stream)) + if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) p_stream->enableExtremes(); }); } @@ -651,7 +651,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns() /// Список столбцов, которых нужно прочитать, чтобы выполнить запрос. Names required_columns = query_analyzer->getRequiredColumns(); - if (query.table && typeid_cast(&*query.table)) + if (query.table && typeid_cast(query.table.get())) { /** Для подзапроса не действуют ограничения на максимальный размер результата. * Так как результат поздапроса - ещё не результат всего запроса. @@ -792,7 +792,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns() transformStreams([&](auto & stream) { - if (IProfilingBlockInputStream * p_stream = dynamic_cast(&*stream)) + if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) { p_stream->setLimits(limits); p_stream->setQuota(quota); @@ -1091,9 +1091,38 @@ void InterpreterSelectQuery::executeLimit() /// Если есть LIMIT if (query.limit_length) { + /** Редкий случай: + * если нет WITH TOTALS и есть подзапрос в FROM, и там на одном из уровней есть WITH TOTALS, + * то при использовании LIMIT-а следует читать данные до конца, а не отменять выполнение запроса раньше, + * потому что при отмене выполнения запроса, мы не получим данные для totals с удалённого сервера. + */ + bool always_read_till_end = false; + if (!query.group_by_with_totals && query.table && typeid_cast(query.table.get())) + { + const ASTSelectQuery * subquery = static_cast(query.table.get()); + + while (subquery->table) + { + if (subquery->group_by_with_totals) + { + /** NOTE Можно ещё проверять, что таблица в подзапросе - распределённая, и что она смотрит только на один шард. + * В остальных случаях totals будет вычислен на сервере-инициаторе запроса, и читать данные до конца не обязательно. + */ + + always_read_till_end = true; + break; + } + + if (typeid_cast(subquery->table.get())) + subquery = static_cast(subquery->table.get()); + else + break; + } + } + transformStreams([&](auto & stream) { - stream = new LimitBlockInputStream(stream, limit_length, limit_offset); + stream = new LimitBlockInputStream(stream, limit_length, limit_offset, always_read_till_end); }); } } diff --git a/dbms/src/Storages/tests/storage_log.cpp b/dbms/src/Storages/tests/storage_log.cpp index 1fdb7391b65..59f61e0ed82 100644 --- a/dbms/src/Storages/tests/storage_log.cpp +++ b/dbms/src/Storages/tests/storage_log.cpp @@ -85,7 +85,7 @@ int main(int argc, char ** argv) DB::WriteBufferFromOStream out_buf(std::cout); - DB::LimitBlockInputStream in_limit(in, 10); + DB::LimitBlockInputStream in_limit(in, 10, 0); DB::RowOutputStreamPtr output_ = new DB::TabSeparatedRowOutputStream(out_buf, sample); DB::BlockOutputStreamFromRowOutputStream output(output_); diff --git a/dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.reference b/dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.reference new file mode 100644 index 00000000000..a594e1495c1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.reference @@ -0,0 +1,3 @@ +1 + +1 diff --git a/dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.sql b/dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.sql new file mode 100644 index 00000000000..1fe3eb3d0d7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00220_with_totals_in_subquery_remote_and_limit.sql @@ -0,0 +1 @@ +SELECT x FROM (SELECT count() AS x FROM remote('127.0.0.1', system.one) WITH TOTALS) LIMIT 1;