From 82c76d846733ae9293fe655e826fd4e568819885 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 13 Feb 2018 22:34:15 +0300 Subject: [PATCH 01/70] added MergeTreePrewhereRangeReader --- .../MergeTreeBaseBlockInputStream.cpp | 365 ++--------- .../MergeTree/MergeTreeBlockReadUtils.h | 8 +- .../MergeTree/MergeTreeRangeReader.cpp | 610 ++++++++++++++++-- .../Storages/MergeTree/MergeTreeRangeReader.h | 161 ++++- .../Storages/MergeTree/MergeTreeReader.cpp | 9 +- dbms/src/Storages/MergeTree/MergeTreeReader.h | 8 +- 6 files changed, 788 insertions(+), 373 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index cb20fc68869..ce3f3b5eba5 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -83,7 +84,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() auto estimateNumRows = [preferred_block_size_bytes, max_block_size_rows, index_granularity, preferred_max_column_in_block_size_bytes, min_filtration_ratio]( - MergeTreeReadTask & task, MergeTreeRangeReader & reader) + MergeTreeReadTask & task, MergeTreePrewhereRangeReader & reader) { if (!task.size_predictor) return max_block_size_rows; @@ -116,315 +117,84 @@ Block MergeTreeBaseBlockInputStream::readFromPart() return index_granularity * granule_to_read - reader.numReadRowsInCurrentGranule(); }; - // read rows from reader and clear columns - auto skipRows = [&estimateNumRows]( - Block & block, MergeTreeRangeReader & reader, MergeTreeReadTask & task, size_t rows) - { - size_t recommended_rows = rows; - if (task.size_predictor) - recommended_rows = std::max(1, estimateNumRows(task, reader)); - while (rows) - { - size_t rows_to_skip = std::min(rows, recommended_rows); - rows -= rows_to_skip; - reader.read(block, rows_to_skip); - for (const auto i : ext::range(0, block.columns())) - { - auto & col = block.getByPosition(i); - if (task.column_name_set.count(col.name)) - { - if (const ColumnArray * column_array = typeid_cast(col.column.get())) - { - /// ColumnArray columns in block could have common offset column, which is used while reading. - /// This is in case of nested data structures. - - /// TODO Very dangerous and unclear. Get rid of this after implemented full-featured Nested data type. - - /// Have to call resize(0) instead of cloneEmpty to save structure. - /// (To keep offsets possibly shared between different arrays.) - static_cast(*column_array->assumeMutable()).getOffsets().resize(0); - /// It's ok until multidimensional arrays are not stored in MergeTree. - static_cast(*column_array->assumeMutable()).getDataPtr() = column_array->getDataPtr()->cloneEmpty(); - } - else - col.column = col.column->cloneEmpty(); - } - } - } - }; - if (prewhere_actions) { do { - /// Let's read the full block of columns needed to calculate the expression in PREWHERE. - MarkRanges ranges_to_read; - /// Last range may be partl read. The same number of rows we need to read after prewhere - size_t rows_was_read_in_last_range = 0; - std::optional pre_range_reader; - - auto processNextRange = [& ranges_to_read, & rows_was_read_in_last_range, & pre_range_reader]( - MergeTreeReadTask & task, MergeTreeReader & pre_reader) + auto processNextRange = [this]() { - auto & range = task.mark_ranges.back(); - pre_range_reader = pre_reader.readRange(range.begin, range.end); - ranges_to_read.push_back(range); - rows_was_read_in_last_range = 0; - task.mark_ranges.pop_back(); + const auto & range = task->mark_ranges.back(); + task->pre_range_reader = pre_reader->readRange( + range.begin, range.end, nullptr, prewhere_actions, + &prewhere_column_name, &task->ordered_names, task->should_reorder); + + task->range_reader = reader->readRange( + range.begin, range.end, &task->pre_range_reader, + nullptr, nullptr, &task->ordered_names, true); + + task->mark_ranges.pop_back(); }; - if (task->current_range_reader) + auto resetRangeReaders = [this]() { - /// Havn't finihsed reading at last step. Copy state for prewhere columns - pre_range_reader = task->current_range_reader->copyForReader(*pre_reader); - if (task->number_of_rows_to_skip) - { - /// number_of_rows_to_skip already was read for prewhere columns. skip them. - pre_range_reader = pre_range_reader->getFutureState(task->number_of_rows_to_skip); - pre_range_reader->disableNextSeek(); - } - } - else - processNextRange(*task, *pre_reader); + task->range_reader.reset(); + task->pre_range_reader.reset(); + }; + + if (!task->range_reader) + processNextRange(); /// FIXME: size prediction model is updated by filtered rows, but it predicts size of unfiltered rows also - size_t recommended_rows = estimateNumRows(*task, *pre_range_reader); + size_t recommended_rows = estimateNumRows(*task, task->range_reader); + if (res && recommended_rows < 1) break; + size_t space_left = std::max(static_cast(1), std::min(max_block_size_rows, recommended_rows)); - while ((pre_range_reader || !task->mark_ranges.empty()) && space_left && !isCancelled()) + size_t total_filtered_rows = 0; + + while (!task->isFinished() && space_left && !isCancelled()) { - if (!pre_range_reader) - processNextRange(*task, *pre_reader); + if (!task->range_reader) + processNextRange(); - size_t rows_to_read = std::min(pre_range_reader->numPendingRows(), space_left); - size_t read_rows = pre_range_reader->read(res, rows_to_read); - rows_was_read_in_last_range += read_rows; - if (pre_range_reader->isReadingFinished()) - pre_range_reader.reset(); + size_t rows_to_read = std::min(task->range_reader.numPendingRows(), space_left); + size_t filtered_rows = 0; - space_left -= read_rows; + auto read_result = task->range_reader.read(res, rows_to_read); + if (task->size_predictor) + { + task->size_predictor->updateFilteredRowsRation( + read_result.getNumAddedRows() + read_result.getNumFilteredRows(), + read_result.getNumFilteredRows()); + } + + total_filtered_rows += filtered_rows; + + if (task->range_reader.isReadingFinished()) + resetRangeReaders(); + + space_left -= rows_to_read; } - /// In case of isCancelled. - if (!res) + if (res.rows() == 0) { - task->current_range_reader.reset(); + res.clear(); return res; } progressImpl({ res.rows(), res.bytes() }); - pre_reader->fillMissingColumns(res, task->ordered_names, task->should_reorder); - /// Compute the expression in PREWHERE. - prewhere_actions->execute(res); - - ColumnPtr prewhere_column = res.getByName(prewhere_column_name).column; - if (task->remove_prewhere_column) + if (task->remove_prewhere_column && res.has(prewhere_column_name)) res.erase(prewhere_column_name); - const auto pre_bytes = res.bytes(); + if (task->size_predictor && res) + task->size_predictor->update(res); - ConstantFilterDescription constant_filter_description(*prewhere_column); - /** If the filter is a constant (for example, it says PREWHERE 0), - * then either return an empty block, or return the block unchanged. - */ - if (constant_filter_description.always_false) - { - /* - If this filter is PREWHERE 0, MergeTree Stream can be marked as done, - and this task can be clear. - If we don't mark this task finished here, readImpl could - jump into endless loop. - Error scenario: - select * from table where isNull(NOT_NULLABLE_COLUMN) AND OTHER PRED; - and isNull pred is promoted to PREWHERE. - (though it is difficult to reproduce) - */ - task->current_range_reader.reset(); - task->mark_ranges.clear(); - res.clear(); - return res; - } - else if (constant_filter_description.always_true) - { - if (task->current_range_reader) - { - if (task->number_of_rows_to_skip) - skipRows(res, *task->current_range_reader, *task, task->number_of_rows_to_skip); - size_t rows_to_read = ranges_to_read.empty() - ? rows_was_read_in_last_range : task->current_range_reader->numPendingRows(); - task->current_range_reader->read(res, rows_to_read); - } + res.checkNumberOfRows(); - for (auto range_idx : ext::range(0, ranges_to_read.size())) - { - const auto & range = ranges_to_read[range_idx]; - task->current_range_reader = reader->readRange(range.begin, range.end); - size_t rows_to_read = range_idx + 1 == ranges_to_read.size() - ? rows_was_read_in_last_range : task->current_range_reader->numPendingRows(); - task->current_range_reader->read(res, rows_to_read); - } - - if (!pre_range_reader) - task->current_range_reader.reset(); - task->number_of_rows_to_skip = 0; - - progressImpl({ 0, res.bytes() - pre_bytes }); - } - else - { - FilterDescription filter_and_holder(*prewhere_column); - - const auto & pre_filter = *filter_and_holder.data; - auto & number_of_rows_to_skip = task->number_of_rows_to_skip; - if (!task->current_range_reader) - number_of_rows_to_skip = 0; - IColumn::Filter post_filter(pre_filter.size()); - - /// Let's read the rest of the columns in the required segments and compose our own filter for them. - size_t pre_filter_pos = 0; - size_t post_filter_pos = 0; - - size_t next_range_idx = 0; - while (pre_filter_pos < pre_filter.size()) - { - if (!task->current_range_reader) - { - if (next_range_idx == ranges_to_read.size()) - throw Exception("Not enough ranges to read after prewhere.", ErrorCodes::LOGICAL_ERROR); - const auto & range = ranges_to_read[next_range_idx++]; - task->current_range_reader = reader->readRange(range.begin, range.end); - } - MergeTreeRangeReader & range_reader = *task->current_range_reader; - size_t current_range_rows_read = 0; - auto pre_filter_begin_pos = pre_filter_pos; - - /// Now we need to read the same number of rows as in prewhere. - size_t rows_to_read = next_range_idx == ranges_to_read.size() - ? rows_was_read_in_last_range : (task->current_range_reader->numPendingRows() - number_of_rows_to_skip); - - auto readRows = [&]() - { - if (pre_filter_pos != pre_filter_begin_pos) - { - /// Fulfilling the promise to read (pre_filter_pos - pre_filter_begin_pos) rows - auto rows = pre_filter_pos - pre_filter_begin_pos; - memcpy(&post_filter[post_filter_pos], &pre_filter[pre_filter_begin_pos], rows); - post_filter_pos += rows; - current_range_rows_read += rows; - if (number_of_rows_to_skip) - { - /** Wasn't able to skip 'number_of_rows_to_skip' with false prewhere conditon - * Just read them and throw away. */ - skipRows(res, range_reader, *task, number_of_rows_to_skip); - number_of_rows_to_skip = 0; - } - range_reader.read(res, rows); - } - }; - - /** (pre_filter_pos - pre_filter_begin_pos) here is the number of rows we promies to read, but - haven't read yet to merge consecutive nonempy granulas. */ - while (current_range_rows_read + (pre_filter_pos - pre_filter_begin_pos) < rows_to_read) - { - auto rows_should_be_copied = pre_filter_pos - pre_filter_begin_pos; - auto range_reader_with_skipped_rows = range_reader.getFutureState(number_of_rows_to_skip + rows_should_be_copied); - auto unread_rows_in_current_granule = range_reader_with_skipped_rows.numPendingRowsInCurrentGranule(); - - const size_t limit = std::min(pre_filter.size(), pre_filter_pos + unread_rows_in_current_granule); - bool will_read_until_mark = unread_rows_in_current_granule == limit - pre_filter_pos; - - UInt8 nonzero = 0; - for (size_t row = pre_filter_pos; row < limit; ++row) - nonzero |= pre_filter[row]; - - if (!nonzero) - { - /// Zero! Prewhere condition is false for all (limit - pre_filter_pos) rows. - readRows(); - - if (will_read_until_mark) - { - /// Can skip the rest of granule with false prewhere conditon right now. - do - { - size_t rows_was_skipped = range_reader.skipToNextMark(); - if (number_of_rows_to_skip < rows_was_skipped) - { - current_range_rows_read += rows_was_skipped - number_of_rows_to_skip; - number_of_rows_to_skip = 0; - } - else - number_of_rows_to_skip -= rows_was_skipped; - } - while (number_of_rows_to_skip); - } - else - { - /// Here reading seems to be done. It's still possible to skip rows during next reading. - number_of_rows_to_skip += limit - pre_filter_pos; - current_range_rows_read += limit - pre_filter_pos; - } - - pre_filter_begin_pos = limit; - } - pre_filter_pos = limit; - } - - readRows(); - - if (next_range_idx != ranges_to_read.size()) - task->current_range_reader.reset(); - } - - if (!pre_range_reader) - task->current_range_reader.reset(); - - if (!post_filter_pos) - { - if (task->size_predictor) - task->size_predictor->updateFilteredRowsRation(pre_filter.size(), pre_filter.size()); - res.clear(); - continue; - } - - progressImpl({ 0, res.bytes() - pre_bytes }); - - post_filter.resize(post_filter_pos); - - /// Filter the columns related to PREWHERE using pre_filter, - /// other columns - using post_filter. - size_t rows = 0; - for (const auto i : ext::range(0, res.columns())) - { - auto & col = res.safeGetByPosition(i); - if (col.name == prewhere_column_name && res.columns() > 1) - continue; - col.column = - col.column->filter(task->column_name_set.count(col.name) ? post_filter : pre_filter, -1); - rows = col.column->size(); - } - if (task->size_predictor) - task->size_predictor->updateFilteredRowsRation(pre_filter.size(), pre_filter.size() - rows); - - /// Replace column with condition value from PREWHERE to a constant. - if (!task->remove_prewhere_column) - res.getByName(prewhere_column_name).column = DataTypeUInt8().createColumnConst(rows, UInt64(1)); - - } - - if (res) - { - if (task->size_predictor) - task->size_predictor->update(res); - - reader->fillMissingColumns(res, task->ordered_names, true); - - res.checkNumberOfRows(); - } } while (!task->isFinished() && !res && !isCancelled()); } @@ -433,37 +203,38 @@ Block MergeTreeBaseBlockInputStream::readFromPart() size_t space_left = std::max(static_cast(1), max_block_size_rows); while (!task->isFinished() && space_left && !isCancelled()) { - if (!task->current_range_reader) + if (!task->range_reader) { auto & range = task->mark_ranges.back(); - task->current_range_reader = reader->readRange(range.begin, range.end); + task->range_reader = reader->readRange(range.begin, range.end, nullptr, + nullptr, nullptr, &task->ordered_names, task->should_reorder); task->mark_ranges.pop_back(); } - size_t rows_to_read = space_left; - size_t recommended_rows = estimateNumRows(*task, *task->current_range_reader); + size_t rows_to_read = std::min(task->range_reader.numPendingRows(), space_left); + size_t recommended_rows = estimateNumRows(*task, task->range_reader); if (res && recommended_rows < 1) break; + rows_to_read = std::min(rows_to_read, std::max(static_cast(1), recommended_rows)); - size_t rows_was_read = task->current_range_reader->read(res, rows_to_read); - if (task->current_range_reader->isReadingFinished()) - task->current_range_reader.reset(); - - if (res && task->size_predictor) + auto read_result = task->range_reader.read(res, rows_to_read); + if (task->size_predictor) { - task->size_predictor->update(res); + task->size_predictor->updateFilteredRowsRation( + read_result.getNumAddedRows() + read_result.getNumFilteredRows(), + read_result.getNumFilteredRows()); } - space_left -= rows_was_read; + if (task->range_reader.isReadingFinished()) + task->range_reader.reset(); + + if (task->size_predictor && res) + task->size_predictor->update(res); + + space_left -= rows_to_read; } - - /// In the case of isCancelled. - if (!res) - return res; - progressImpl({ res.rows(), res.bytes() }); - reader->fillMissingColumns(res, task->ordered_names, task->should_reorder); } return res; diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index b694b4535da..c93452e0bce 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -47,12 +47,10 @@ struct MergeTreeReadTask /// Used to satistfy preferred_block_size_bytes limitation MergeTreeBlockSizePredictorPtr size_predictor; /// used to save current range processing status - std::optional current_range_reader; - /// the number of rows wasn't read by range_reader if condition in prewhere was false - /// helps to skip graunule if all conditions will be aslo false - size_t number_of_rows_to_skip; + MergeTreePrewhereRangeReader range_reader; + MergeTreePrewhereRangeReader pre_range_reader; - bool isFinished() const { return mark_ranges.empty() && !current_range_reader; } + bool isFinished() const { return mark_ranges.empty() && !range_reader; } MergeTreeReadTask( const MergeTreeData::DataPartPtr & data_part, const MarkRanges & mark_ranges, const size_t part_index_in_query, diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 3e6e4927829..2c8ed553328 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -1,70 +1,598 @@ #include +#include +#include +#include + +#if __SSE2__ +#include +#endif namespace DB { -MergeTreeRangeReader::MergeTreeRangeReader( - MergeTreeReader & merge_tree_reader, size_t from_mark, size_t to_mark, size_t index_granularity) - : merge_tree_reader(merge_tree_reader), current_mark(from_mark), last_mark(to_mark) - , index_granularity(index_granularity) +MergeTreePrewhereRangeReader::DelayedStream::DelayedStream( + size_t from_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader) + : current_mark(from_mark), current_offset(0), num_delayed_rows(0) + , index_granularity(index_granularity), merge_tree_reader(merge_tree_reader) + , continue_reading(false), is_finished(false) { } -size_t MergeTreeRangeReader::skipToNextMark() +size_t MergeTreePrewhereRangeReader::DelayedStream::position() const { - auto unread_rows_in_current_part = numPendingRowsInCurrentGranule(); - continue_reading = false; - ++current_mark; - if (current_mark == last_mark) - is_reading_finished = true; - read_rows_after_current_mark = 0; - return unread_rows_in_current_part; + return current_mark * index_granularity + current_offset + num_delayed_rows; } -MergeTreeRangeReader MergeTreeRangeReader::getFutureState(size_t rows_to_read) const + +size_t MergeTreePrewhereRangeReader::DelayedStream::readRows(Block & block, size_t num_rows) { - MergeTreeRangeReader copy = *this; - copy.read_rows_after_current_mark += rows_to_read; - size_t read_parts = copy.read_rows_after_current_mark / index_granularity; - copy.current_mark += read_parts; - copy.read_rows_after_current_mark -= index_granularity * read_parts; - return copy; + if (num_rows) + { + size_t rows_read = merge_tree_reader->readRows(current_mark, continue_reading, num_rows, block); + continue_reading = true; + + /// Zero rows_read my be either because reading has finished + /// or because there is no columns we can read in current part (for example, all columns are default). + /// In the last case we can't finish reading, but it's also ok for the first case + /// because we can finish reading by calculation the number of pending rows. + if (0 < rows_read && rows_read < num_rows) + is_finished = true; + + return rows_read; + } + + return 0; } -size_t MergeTreeRangeReader::read(Block & res, size_t max_rows_to_read) +size_t MergeTreePrewhereRangeReader::DelayedStream::read(Block & block, size_t from_mark, size_t offset, size_t num_rows) { - size_t rows_to_read = numPendingRows(); - rows_to_read = std::min(rows_to_read, max_rows_to_read); - if (rows_to_read == 0) - throw Exception("Logical error: 0 rows to read.", ErrorCodes::LOGICAL_ERROR); + if (position() == from_mark * index_granularity + offset) + { + num_delayed_rows += num_rows; + return 0; + } + else + { + size_t read_rows = finalize(block); - auto read_rows = merge_tree_reader.get().readRows(current_mark, continue_reading, rows_to_read, res); + continue_reading = false; + current_mark = from_mark; + current_offset = offset; + num_delayed_rows = num_rows; - if (read_rows && read_rows < rows_to_read) - is_reading_finished = true; + return read_rows; + } +} - if (!read_rows) - read_rows = rows_to_read; +size_t MergeTreePrewhereRangeReader::DelayedStream::finalize(Block & block) +{ + if (current_offset && !continue_reading) + { + size_t granules_to_skip = current_offset / index_granularity; + current_mark += granules_to_skip; + current_offset -= granules_to_skip * index_granularity; - continue_reading = true; + if (current_offset) + { + Block temp_block; + readRows(temp_block, current_offset); + } + } - read_rows_after_current_mark += read_rows; - size_t read_parts = read_rows_after_current_mark / index_granularity; - current_mark += read_parts; - read_rows_after_current_mark -= index_granularity * read_parts; + size_t rows_to_read = num_delayed_rows; + current_offset += num_delayed_rows; + num_delayed_rows = 0; - if (current_mark == last_mark) - is_reading_finished = true; + return readRows(block, rows_to_read); +} + +MergeTreePrewhereRangeReader::Stream::Stream(size_t from_mark, size_t to_mark, size_t index_granularity, + MergeTreeReader * merge_tree_reader) + : current_mark(from_mark), offset_after_current_mark(0) + , index_granularity(index_granularity), last_mark(to_mark) + , stream(from_mark, index_granularity, merge_tree_reader) +{ +} + +void MergeTreePrewhereRangeReader::Stream::checkNotFinished() const +{ + if (isFinished()) + throw Exception("Cannot read out of marks range.", ErrorCodes::LOGICAL_ERROR); +} + +void MergeTreePrewhereRangeReader::Stream::checkEnoughSpaceInCurrentGranula(size_t num_rows) const +{ + if (num_rows + offset_after_current_mark > index_granularity) + throw Exception("Cannot read from granule more than index_granularity.", ErrorCodes::LOGICAL_ERROR); +} + +size_t MergeTreePrewhereRangeReader::Stream::readRows(Block & block, size_t num_rows) +{ + size_t rows_read = stream.read(block, current_mark, offset_after_current_mark, num_rows); + + if (stream.isFinished()) + finish(); + + return rows_read; +} + +size_t MergeTreePrewhereRangeReader::Stream::read(Block & block, size_t num_rows, + bool skip_remaining_rows_in_current_granule) +{ + checkEnoughSpaceInCurrentGranula(num_rows); + + if (num_rows) + { + checkNotFinished(); + + size_t read_rows = readRows(block, num_rows); + offset_after_current_mark += num_rows; + + if (offset_after_current_mark == index_granularity || skip_remaining_rows_in_current_granule) + { + /// Start new granule; skipped_rows_after_offset is already zero. + ++current_mark; + offset_after_current_mark = 0; + } + + return read_rows; + } + else + { + /// Nothing to read. + if (skip_remaining_rows_in_current_granule) + { + /// Skip the rest of the rows in granule and start new one. + checkNotFinished(); + + ++current_mark; + offset_after_current_mark = 0; + } + + return 0; + } +} + +void MergeTreePrewhereRangeReader::Stream::skip(size_t num_rows) +{ + if (num_rows) + { + checkNotFinished(); + checkEnoughSpaceInCurrentGranula(num_rows); + + offset_after_current_mark += num_rows; + + if (offset_after_current_mark == index_granularity) + { + /// Start new granule; skipped_rows_after_offset is already zero. + ++current_mark; + offset_after_current_mark = 0; + } + } +} + +size_t MergeTreePrewhereRangeReader::Stream::finalize(Block & block) +{ + size_t read_rows = stream.finalize(block); + + if (stream.isFinished()) + finish(); return read_rows; } -MergeTreeRangeReader MergeTreeRangeReader::copyForReader(MergeTreeReader & reader) + +void MergeTreePrewhereRangeReader::ReadResult::addGranule(size_t num_rows) { - MergeTreeRangeReader copy(reader, current_mark, last_mark, index_granularity); - copy.continue_reading = continue_reading; - copy.read_rows_after_current_mark = read_rows_after_current_mark; - return copy; + rows_per_granule.push_back(num_rows); + num_read_rows += num_rows; +} + +void MergeTreePrewhereRangeReader::ReadResult::adjustLastGranule(size_t num_rows_to_subtract) +{ + if (rows_per_granule.empty()) + throw Exception("Can't adjust last granule because no granules were added.", ErrorCodes::LOGICAL_ERROR); + + if (num_rows_to_subtract > rows_per_granule.back()) + throw Exception("Can't adjust last granule because it has " + toString(rows_per_granule.back()) + + "rows, but try to subtract " + toString(num_rows_to_subtract) + " rows.", + ErrorCodes::LOGICAL_ERROR); + + rows_per_granule.back() -= num_rows_to_subtract; + num_read_rows -= num_rows_to_subtract; +} + +void MergeTreePrewhereRangeReader::ReadResult::clear() +{ + /// Need to save information about the number of granules. + rows_per_granule.assign(rows_per_granule.size(), 0); + num_filtered_rows += num_read_rows - num_zeros_in_filter; + num_read_rows = 0; + num_added_rows = 0; + num_zeros_in_filter = 0; + filter = nullptr; +} + +void MergeTreePrewhereRangeReader::ReadResult::optimize() +{ + if (num_read_rows == 0 || !filter) + return; + + ConstantFilterDescription constant_filter_description(*filter); + + if (constant_filter_description.always_false) + clear(); + else if (constant_filter_description.always_true) + filter = nullptr; + else + { + ColumnPtr prev_filter = std::move(filter); + FilterDescription prev_description(*prev_filter); + + MutableColumnPtr new_filter_ptr = ColumnUInt8::create(prev_description.data->size()); + auto & new_filter = static_cast(*new_filter_ptr); + IColumn::Filter & new_data = new_filter.getData(); + + collapseZeroTails(*prev_description.data, new_data); + + size_t num_removed_zeroes = new_filter.size() - num_read_rows; + num_read_rows = new_filter.size(); + num_zeros_in_filter -= num_removed_zeroes; + + filter = std::move(new_filter_ptr); + } +} + +void MergeTreePrewhereRangeReader::ReadResult::collapseZeroTails(const IColumn::Filter & filter, + IColumn::Filter & new_filter) +{ + auto filter_data = filter.data(); + auto new_filter_data = new_filter.data(); + + size_t rows_in_filter_from_prev_iteration = filter.size() - num_read_rows; + if (rows_in_filter_from_prev_iteration) + { + memcpySmallAllowReadWriteOverflow15(new_filter_data, filter_data, rows_in_filter_from_prev_iteration); + filter_data += rows_in_filter_from_prev_iteration; + new_filter_data += rows_in_filter_from_prev_iteration; + } + + for (auto & rows_to_read : rows_per_granule) + { + /// Count the number of zeros at the end of filter for rows were read from current granule. + size_t filtered_rows_num_at_granule_end = numZerosInTail(filter_data, filter_data + rows_to_read); + rows_to_read -= filtered_rows_num_at_granule_end; + + memcpySmallAllowReadWriteOverflow15(new_filter_data, filter_data, rows_to_read); + filter_data += rows_to_read; + new_filter_data += rows_to_read; + + filter_data += filtered_rows_num_at_granule_end; + } + + new_filter.resize(new_filter_data - new_filter.data()); +} + + +size_t MergeTreePrewhereRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, const UInt8 * end) +{ + size_t count = 0; + +#if __SSE2__ && __POPCNT__ + const __m128i zero16 = _mm_setzero_si128(); + while (end - begin >= 64) + { + end -= 64; + auto pos = end; + UInt64 val = + static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(pos)), + zero16))) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(pos + 16)), + zero16))) << 16) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(pos + 32)), + zero16))) << 32) + | (static_cast(_mm_movemask_epi8(_mm_cmpgt_epi8( + _mm_loadu_si128(reinterpret_cast(pos + 48)), + zero16))) << 48); + if (val == 0) + count += 64; + else + { + count += __builtin_clzll(val); + return count; + } + } +#endif + + while (end > begin && *(--end) == 0) + { + ++count; + } + return count; +} + +size_t MergeTreePrewhereRangeReader::ReadResult::numZerosInFilter() const +{ + if (!filter) + return 0; + + { + ConstantFilterDescription constant_filter_description(*filter); + if (constant_filter_description.always_false) + return filter->size(); + if (constant_filter_description.always_true) + return 0; + } + + FilterDescription description(*filter); + + auto data = description.data; + auto size = description.data->size(); + + return size - countBytesInFilter(*data); +} + + +void MergeTreePrewhereRangeReader::ReadResult::setFilter(ColumnPtr filter_) +{ + if (!filter_ && filter) + throw Exception("Can't remove exising filter with empty.", ErrorCodes::LOGICAL_ERROR); + + if (!filter_) + return; + + if (filter_->size() < num_read_rows) + throw Exception("Can't set filter because it's size is " + toString(filter_->size()) + " but " + + toString(num_read_rows) + " rows was read.", ErrorCodes::LOGICAL_ERROR); + + if (filter && filter_->size() != filter->size()) + throw Exception("Can't set filter because it's size is " + toString(filter_->size()) + " but previous filter" + + " has size " + toString(filter->size()) + ".", ErrorCodes::LOGICAL_ERROR); + + filter = std::move(filter_); + size_t num_zeros = numZerosInFilter(); + + if (num_zeros < num_zeros_in_filter) + throw Exception("New filter has less zeros than previous.", ErrorCodes::LOGICAL_ERROR); + + size_t added_zeros = num_zeros - num_zeros_in_filter; + num_added_rows -= added_zeros; + num_filtered_rows += added_zeros; + num_zeros_in_filter = num_zeros; +} + +MergeTreePrewhereRangeReader::MergeTreePrewhereRangeReader( + MergeTreePrewhereRangeReader * prev_reader, MergeTreeReader * merge_tree_reader, + size_t from_mark, size_t to_mark, size_t index_granularity, + ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, + const Names * ordered_names, bool always_reorder) + : stream(from_mark, to_mark, index_granularity, merge_tree_reader) + , prev_reader(prev_reader), prewhere_actions(std::move(prewhere_actions)) + , prewhere_column_name(prewhere_column_name), ordered_names(ordered_names), always_reorder(always_reorder) +{ +} + + +MergeTreePrewhereRangeReader::ReadResult MergeTreePrewhereRangeReader::read( + Block & res, size_t max_rows) +{ + if (max_rows == 0) + throw Exception("Expected at least 1 row to read, got 0.", ErrorCodes::LOGICAL_ERROR); + + if (max_rows > numPendingRows()) + throw Exception("Want to read " + toString(max_rows) + " rows, but has only " + + toString(numPendingRows()) + " pending rows.", ErrorCodes::LOGICAL_ERROR); + + ReadResult read_result; + + if (prev_reader) + read_result = prev_reader->read(res, max_rows); + + readRows(res, max_rows, read_result); + + if (!res) + return read_result; + + executePrewhereActionsAndFilterColumns(res, read_result); + return read_result; +} + +void MergeTreePrewhereRangeReader::readRows(Block & block, size_t max_rows, ReadResult & result) +{ + if (prev_reader && result.numReadRows() == 0) + { + /// If zero rows were read on prev step, than there is no more rows to read. + /// Last granule may have less rows than index_granularity, so finish reading manually. + stream.finish(); + return; + } + + size_t rows_to_skip_in_last_granule = 0; + + if (!result.rowsPerGranule().empty()) + { + size_t rows_in_last_granule = result.rowsPerGranule().back(); + result.optimize(); + rows_to_skip_in_last_granule = rows_in_last_granule - result.rowsPerGranule().back(); + + if (auto & filter = result.getFilter()) + { + if (ConstantFilterDescription(*filter).always_false) + throw Exception("Shouldn't read rows with constant zero prewhere result.", ErrorCodes::LOGICAL_ERROR); + } + } + + if (result.rowsPerGranule().empty()) + { + /// Stream is lazy. result.num_added_rows is the number of rows added to block which is not equal to + /// result.num_rows_read until call to stream.finalize(). Also result.num_added_rows may be less than + /// result.num_rows_read if the last granule in range also the last in part (so we have to adjust last granule). + { + size_t space_left = max_rows; + while (space_left && !stream.isFinished()) + { + auto rows_to_read = std::min(space_left, stream.numPendingRowsInCurrentGranule()); + bool last = rows_to_read == space_left; + result.addRows(stream.read(block, rows_to_read, !last)); + result.addGranule(rows_to_read); + space_left -= rows_to_read; + } + } + + stream.skip(rows_to_skip_in_last_granule); + result.addRows(stream.finalize(block)); + + auto last_granule = result.rowsPerGranule().back(); + + auto added_rows =result.getNumAddedRows(); + + if (max_rows - last_granule > added_rows) + throw Exception("RangeReader expected reading of at least " + toString(max_rows - last_granule) + + " rows, but only " + toString(added_rows) + " was read.", ErrorCodes::LOGICAL_ERROR); + + /// Last granule may be incomplete. + size_t adjustment = max_rows - added_rows; + result.adjustLastGranule(adjustment); + + } + else + { + size_t added_rows = 0; + auto & rows_per_granule = result.rowsPerGranule(); + + auto size = rows_per_granule.size(); + for (auto i : ext::range(0, size)) + { + bool last = i + 1 == size; + added_rows += stream.read(block, rows_per_granule[i], !last); + } + + stream.skip(rows_to_skip_in_last_granule); + added_rows += stream.finalize(block); + + /// added_rows may be zero if all columns were read in prewhere and it's ok. + if (added_rows && added_rows != result.numReadRows()) + throw Exception("RangeReader read " + toString(added_rows) + " rows, but " + + toString(result.numReadRows()) + " expected.", ErrorCodes::LOGICAL_ERROR); + } +} + +void MergeTreePrewhereRangeReader::executePrewhereActionsAndFilterColumns(Block & block, ReadResult & result) +{ + + const auto & columns = stream.reader()->getColumns(); + + auto filterColumns = [&block, &columns](const IColumn::Filter & filter) + { + for (const auto & column : columns) + { + if (block.has(column.name)) + { + auto & column_with_type_and_name = block.getByName(column.name); + column_with_type_and_name.column = std::move(column_with_type_and_name.column)->filter(filter, -1); + } + } + }; + + auto filterBlock = [&block](const IColumn::Filter & filter) + { + for (const auto i : ext::range(0, block.columns())) + { + auto & col = block.safeGetByPosition(i); + + if (col.column && col.column->size() == filter.size()) + col.column = std::move(col.column)->filter(filter, -1); + } + }; + + if (auto & filter = result.getFilter()) + { + ConstantFilterDescription constant_filter_description(*filter); + if (constant_filter_description.always_false) + throw Exception("RangeReader mustn't execute prewhere actions with const zero prewhere result.", + ErrorCodes::LOGICAL_ERROR); + if (!constant_filter_description.always_true) + { + FilterDescription filter_and_holder(*filter); + filterColumns(*filter_and_holder.data); + } + } + + if (!columns.empty()) + { + if (columns.size() == block.columns()) + { + stream.reader()->fillMissingColumns(block, *ordered_names, always_reorder); + + if (prewhere_actions) + prewhere_actions->execute(block); + } + else + { + /// Columns in block may have different size here. Create temporary block which has only read columns. + Block tmp_block; + for (const auto & column : columns) + { + if (block.has(column.name)) + { + auto & column_with_type_and_name = block.getByName(column.name); + tmp_block.insert(column_with_type_and_name); + column_with_type_and_name.column = nullptr; + } + } + + if (tmp_block) + stream.reader()->fillMissingColumns(tmp_block, *ordered_names, always_reorder); + + if (prewhere_actions) + prewhere_actions->execute(tmp_block); + + for (auto col_num : ext::range(0, block.columns())) + { + auto & column = block.getByPosition(col_num); + if (!tmp_block.has(column.name)) + tmp_block.insert(std::move(column)); + } + + std::swap(block, tmp_block); + } + } + + ColumnPtr filter; + if (prewhere_actions) + { + auto & prewhere_column = block.getByName(*prewhere_column_name); + + ConstantFilterDescription constant_filter_description(*prewhere_column.column); + if (constant_filter_description.always_false) + { + result.clear(); + block.clear(); + return; + } + else if (!constant_filter_description.always_true) + { + filter = std::move(prewhere_column.column); + FilterDescription filter_and_holder(*filter); + filterBlock(*filter_and_holder.data); + } + + prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), UInt64(1)); + } + + if (filter && result.getFilter()) + { + /// TODO: implement for prewhere chain. + /// In order to do it we need combine filter and result.filter, where filter filters only '1' in result.filter. + throw Exception("MergeTreePrewhereRangeReader chain with several prewhere actions in not implemented.", + ErrorCodes::LOGICAL_ERROR); + } + + if (filter) + result.setFilter(filter); } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 0d8b5357c28..1d8601fe573 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -10,43 +10,152 @@ class MergeTreeReader; /// MergeTreeReader iterator which allows sequential reading for arbitrary number of rows between pairs of marks in the same part. /// Stores reading state, which can be inside granule. Can skip rows in current granule and start reading from next mark. /// Used generally for reading number of rows less than index granularity to decrease cache misses for fat blocks. -class MergeTreeRangeReader +class MergeTreePrewhereRangeReader { public: - size_t numPendingRows() const { return (last_mark - current_mark) * index_granularity - read_rows_after_current_mark; } - size_t numPendingRowsInCurrentGranule() const { return index_granularity - read_rows_after_current_mark; } + MergeTreePrewhereRangeReader(MergeTreePrewhereRangeReader * prev_reader, MergeTreeReader * merge_tree_reader, + size_t from_mark, size_t to_mark, size_t index_granularity, + ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, + const Names * ordered_names, bool always_reorder); - size_t numReadRowsInCurrentGranule() const { return read_rows_after_current_mark; } + MergeTreePrewhereRangeReader() : is_initialized(false) { } - /// Seek to next mark before next reading. - size_t skipToNextMark(); - /// Seturn state will be afrer reading rows_to_read, no reading happens. - MergeTreeRangeReader getFutureState(size_t rows_to_read) const; + bool isReadingFinished() const { return prev_reader ? prev_reader->isReadingFinished() : stream.isFinished(); } - /// If columns are not present in the block, adds them. If they are present - appends the values that have been read. - /// Do not add columns, if the files are not present for them. - /// Block should contain either no columns from the columns field, or all columns for which files are present. - /// Returns the number of rows was read. - size_t read(Block & res, size_t max_rows_to_read); + size_t numReadRowsInCurrentGranule() const { return prev_reader ? prev_reader->numReadRowsInCurrentGranule() : stream.numReadRowsInCurrentGranule(); } + size_t numPendingRowsInCurrentGranule() const { return prev_reader ? prev_reader->numPendingRowsInCurrentGranule() : stream.numPendingRowsInCurrentGranule(); } + size_t numPendingRows() const { return prev_reader ? prev_reader->numPendingRows() : stream.numPendingRows(); } - bool isReadingFinished() const { return is_reading_finished; } + operator bool() const { return is_initialized; } + void reset() { is_initialized = false; } - void disableNextSeek() { continue_reading = true; } - /// Return the same state for other MergeTreeReader. - MergeTreeRangeReader copyForReader(MergeTreeReader & reader); + class DelayedStream + { + public: + DelayedStream() {} + DelayedStream(size_t from_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader); + + /// Returns the number of rows added to block. + /// NOTE: have to return number of rows because block has broken invariant: + /// some columns may have different size (for example, default columns may be zero size). + size_t read(Block & block, size_t from_mark, size_t offset, size_t num_rows); + size_t finalize(Block & block); + + bool isFinished() const { return is_finished; } + + MergeTreeReader * reader() const { return merge_tree_reader; } + + private: + size_t current_mark; + size_t current_offset; + size_t num_delayed_rows; + + size_t index_granularity; + MergeTreeReader * merge_tree_reader; + bool continue_reading; + bool is_finished; + + size_t position() const; + size_t readRows(Block & block, size_t num_rows); + }; + + class Stream + { + + public: + Stream() {} + Stream(size_t from_mark, size_t to_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader); + + /// Returns the n + size_t read(Block & block, size_t num_rows, bool skip_remaining_rows_in_current_granule); + size_t finalize(Block & block); + void skip(size_t num_rows); + + void finish() { current_mark = last_mark; } + bool isFinished() const { return current_mark >= last_mark; } + + size_t numReadRowsInCurrentGranule() const { return offset_after_current_mark; } + size_t numPendingRowsInCurrentGranule() const { return index_granularity - numReadRowsInCurrentGranule(); } + size_t numRendingGranules() const { return last_mark - current_mark; } + size_t numPendingRows() const { return numRendingGranules() * index_granularity - offset_after_current_mark; } + + MergeTreeReader * reader() const { return stream.reader(); } + + private: + size_t current_mark; + /// Invariant: offset_after_current_mark + skipped_rows_after_offset < index_granularity + size_t offset_after_current_mark; + + size_t index_granularity; + size_t last_mark; + + DelayedStream stream; + + void checkNotFinished() const; + void checkEnoughSpaceInCurrentGranula(size_t num_rows) const; + size_t readRows(Block & block, size_t num_rows); + }; + + /// Statistics after next reading step. + class ReadResult + { + public: + const std::vector & rowsPerGranule() const { return rows_per_granule; } + /// The number of rows were read at LAST iteration in chain. <= num_added_rows + num_filtered_rows. + size_t numReadRows() const { return num_read_rows; } + /// The number of rows were added to block as a result of reading chain. + size_t getNumAddedRows() const { return num_added_rows; } + /// The number of filtered rows at all steps in reading chain. + size_t getNumFilteredRows() const { return num_filtered_rows; } + /// Filter you need to allply to newly-read columns in order to add them to block. + const ColumnPtr & getFilter() const { return filter; } + + void addGranule(size_t num_rows); + void adjustLastGranule(size_t num_rows_to_subtract); + void addRows(size_t rows) { num_added_rows += rows; } + + /// Set filter or replace old one. Filter must have more zeroes than previous. + void setFilter(ColumnPtr filter_); + /// For each granule calculate the number of filtered rows at the end. Remove them and update filter. + void optimize(); + /// Remove all rows from granules. + void clear(); + + private: + /// The number of rows read from each granule. + std::vector rows_per_granule; + /// Sum(rows_per_granule) + size_t num_read_rows = 0; + /// The number of rows was added to block while reading columns. May be zero if no read columns present in part. + size_t num_added_rows = 0; + /// num_zeros_in_filter + the number of rows removed after optimizes. + size_t num_filtered_rows = 0; + /// Zero if filter is nullptr. + size_t num_zeros_in_filter = 0; + /// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= total_rows_read. + ColumnPtr filter; + + void collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter); + size_t numZerosInFilter() const; + static size_t numZerosInTail(const UInt8 * begin, const UInt8 * end); + }; + + ReadResult read(Block & res, size_t max_rows); private: - MergeTreeRangeReader(MergeTreeReader & merge_tree_reader, size_t from_mark, size_t to_mark, size_t index_granularity); - std::reference_wrapper merge_tree_reader; - size_t current_mark; - size_t last_mark; - size_t read_rows_after_current_mark = 0; - size_t index_granularity; - bool continue_reading = false; - bool is_reading_finished = false; + void readRows(Block & block, size_t max_rows, ReadResult & result); + void executePrewhereActionsAndFilterColumns(Block & block, ReadResult & result); - friend class MergeTreeReader; + Stream stream; + MergeTreePrewhereRangeReader * prev_reader; /// If not nullptr, read from prev_reader firstly. + + ExpressionActionsPtr prewhere_actions; /// If not nullptr, calculate filter. + const String * prewhere_column_name; + const Names * ordered_names; + bool always_reorder; + + bool is_initialized = true; }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 4ae50b00b2e..73473192ce8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -66,9 +66,14 @@ const MergeTreeReader::ValueSizeMap & MergeTreeReader::getAvgValueSizeHints() co } -MergeTreeRangeReader MergeTreeReader::readRange(size_t from_mark, size_t to_mark) +MergeTreePrewhereRangeReader MergeTreeReader::readRange( + size_t from_mark, size_t to_mark, MergeTreePrewhereRangeReader * prev_reader, + ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, + const Names * ordered_names, bool always_reorder) { - return MergeTreeRangeReader(*this, from_mark, to_mark, storage.index_granularity); + return MergeTreePrewhereRangeReader( + prev_reader, this, from_mark, to_mark, storage.index_granularity, + prewhere_actions, prewhere_column_name, ordered_names, always_reorder); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 6f4c1f60dc2..a9d384ce8ef 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -39,13 +39,17 @@ public: const ValueSizeMap & getAvgValueSizeHints() const; /// Create MergeTreeRangeReader iterator, which allows reading arbitrary number of rows from range. - MergeTreeRangeReader readRange(size_t from_mark, size_t to_mark); + MergeTreePrewhereRangeReader readRange(size_t from_mark, size_t to_mark, MergeTreePrewhereRangeReader * prev_reader, + ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, + const Names * ordered_names, bool always_reorder); /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// If at least one column was added, reorders all columns in the block according to ordered_names. void fillMissingColumns(Block & res, const Names & ordered_names, const bool always_reorder = false); + const NamesAndTypesList & getColumns() const { return columns; } + private: class Stream { @@ -117,7 +121,7 @@ private: /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res); - friend class MergeTreeRangeReader; + friend class MergeTreePrewhereRangeReader::DelayedStream; }; } From ce70d4faa9797512c144e1c57c0bde70c6048484 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 14:45:58 +0300 Subject: [PATCH 02/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- .../MergeTreeBaseBlockInputStream.cpp | 153 +++------- .../MergeTree/MergeTreeBlockReadUtils.h | 8 +- .../MergeTree/MergeTreeRangeReader.cpp | 289 +++++++++--------- .../Storages/MergeTree/MergeTreeRangeReader.h | 81 +++-- .../Storages/MergeTree/MergeTreeReader.cpp | 11 - dbms/src/Storages/MergeTree/MergeTreeReader.h | 7 +- 6 files changed, 235 insertions(+), 314 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index ce3f3b5eba5..1cb567a0caf 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { extern const int ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER; - extern const int LOGICAL_ERROR; } @@ -84,7 +83,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() auto estimateNumRows = [preferred_block_size_bytes, max_block_size_rows, index_granularity, preferred_max_column_in_block_size_bytes, min_filtration_ratio]( - MergeTreeReadTask & task, MergeTreePrewhereRangeReader & reader) + MergeTreeReadTask & task, MergeTreeRangeReader & reader) { if (!task.size_predictor) return max_block_size_rows; @@ -102,7 +101,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() size_t rows_to_read_for_max_size_column = task.size_predictor->estimateNumRowsForMaxSizeColumn(preferred_max_column_in_block_size_bytes); double filtration_ratio = std::max(min_filtration_ratio, 1.0 - task.size_predictor->filtered_rows_ratio); - size_t rows_to_read_for_max_size_column_with_filtration + auto rows_to_read_for_max_size_column_with_filtration = static_cast(rows_to_read_for_max_size_column / filtration_ratio); /// If preferred_max_column_in_block_size_bytes is used, number of rows to read can be less than index_granularity. @@ -117,127 +116,49 @@ Block MergeTreeBaseBlockInputStream::readFromPart() return index_granularity * granule_to_read - reader.numReadRowsInCurrentGranule(); }; - if (prewhere_actions) + if (!task->range_reader.isInitialized()) { - do + if (prewhere_actions) { - auto processNextRange = [this]() - { - const auto & range = task->mark_ranges.back(); - task->pre_range_reader = pre_reader->readRange( - range.begin, range.end, nullptr, prewhere_actions, - &prewhere_column_name, &task->ordered_names, task->should_reorder); - - task->range_reader = reader->readRange( - range.begin, range.end, &task->pre_range_reader, - nullptr, nullptr, &task->ordered_names, true); - - task->mark_ranges.pop_back(); - }; - - auto resetRangeReaders = [this]() - { - task->range_reader.reset(); - task->pre_range_reader.reset(); - }; - - if (!task->range_reader) - processNextRange(); - - /// FIXME: size prediction model is updated by filtered rows, but it predicts size of unfiltered rows also - size_t recommended_rows = estimateNumRows(*task, task->range_reader); - - if (res && recommended_rows < 1) - break; - - size_t space_left = std::max(static_cast(1), std::min(max_block_size_rows, recommended_rows)); - - size_t total_filtered_rows = 0; - - while (!task->isFinished() && space_left && !isCancelled()) - { - if (!task->range_reader) - processNextRange(); - - size_t rows_to_read = std::min(task->range_reader.numPendingRows(), space_left); - size_t filtered_rows = 0; - - auto read_result = task->range_reader.read(res, rows_to_read); - if (task->size_predictor) - { - task->size_predictor->updateFilteredRowsRation( - read_result.getNumAddedRows() + read_result.getNumFilteredRows(), - read_result.getNumFilteredRows()); - } - - total_filtered_rows += filtered_rows; - - if (task->range_reader.isReadingFinished()) - resetRangeReaders(); - - space_left -= rows_to_read; - } - - if (res.rows() == 0) - { - res.clear(); - return res; - } - - progressImpl({ res.rows(), res.bytes() }); - - if (task->remove_prewhere_column && res.has(prewhere_column_name)) - res.erase(prewhere_column_name); - - if (task->size_predictor && res) - task->size_predictor->update(res); - - - res.checkNumberOfRows(); + task->pre_range_reader = MergeTreeRangeReader( + pre_reader.get(), index_granularity, nullptr, prewhere_actions, + &prewhere_column_name, &task->ordered_names, task->should_reorder); + task->range_reader = MergeTreeRangeReader( + reader.get(), index_granularity, &task->pre_range_reader, nullptr, nullptr, nullptr, true); } - while (!task->isFinished() && !res && !isCancelled()); - } - else - { - size_t space_left = std::max(static_cast(1), max_block_size_rows); - while (!task->isFinished() && space_left && !isCancelled()) + else { - if (!task->range_reader) - { - auto & range = task->mark_ranges.back(); - task->range_reader = reader->readRange(range.begin, range.end, nullptr, - nullptr, nullptr, &task->ordered_names, task->should_reorder); - task->mark_ranges.pop_back(); - } - - size_t rows_to_read = std::min(task->range_reader.numPendingRows(), space_left); - size_t recommended_rows = estimateNumRows(*task, task->range_reader); - if (res && recommended_rows < 1) - break; - - rows_to_read = std::min(rows_to_read, std::max(static_cast(1), recommended_rows)); - - auto read_result = task->range_reader.read(res, rows_to_read); - if (task->size_predictor) - { - task->size_predictor->updateFilteredRowsRation( - read_result.getNumAddedRows() + read_result.getNumFilteredRows(), - read_result.getNumFilteredRows()); - } - - if (task->range_reader.isReadingFinished()) - task->range_reader.reset(); - - if (task->size_predictor && res) - task->size_predictor->update(res); - - space_left -= rows_to_read; + task->range_reader = MergeTreeRangeReader( + reader.get(), index_granularity, nullptr, nullptr, + nullptr, &task->ordered_names, task->should_reorder); } - progressImpl({ res.rows(), res.bytes() }); } - return res; + size_t recommended_rows = estimateNumRows(*task, task->range_reader); + size_t rows_to_read = std::max(static_cast(1), + std::min(max_block_size_rows, recommended_rows)); + + auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges); + + progressImpl({ read_result.block.rows(), read_result.block.bytes() }); + + if (task->size_predictor) + { + task->size_predictor->updateFilteredRowsRation( + read_result.getNumAddedRows() + read_result.getNumFilteredRows(), + read_result.getNumFilteredRows()); + + if (read_result.block) + task->size_predictor->update(read_result.block); + } + + if (task->remove_prewhere_column && res.has(prewhere_column_name)) + res.erase(prewhere_column_name); + + res.checkNumberOfRows(); + + return read_result.block; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h index c93452e0bce..51ffae2b848 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockReadUtils.h @@ -46,11 +46,11 @@ struct MergeTreeReadTask const bool should_reorder; /// Used to satistfy preferred_block_size_bytes limitation MergeTreeBlockSizePredictorPtr size_predictor; - /// used to save current range processing status - MergeTreePrewhereRangeReader range_reader; - MergeTreePrewhereRangeReader pre_range_reader; + /// Used to save current range processing status + MergeTreeRangeReader range_reader; + MergeTreeRangeReader pre_range_reader; - bool isFinished() const { return mark_ranges.empty() && !range_reader; } + bool isFinished() const { return mark_ranges.empty() && range_reader.isCurrentRangeFinished(); } MergeTreeReadTask( const MergeTreeData::DataPartPtr & data_part, const MarkRanges & mark_ranges, const size_t part_index_in_query, diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 2c8ed553328..974db6131ee 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -10,7 +10,7 @@ namespace DB { -MergeTreePrewhereRangeReader::DelayedStream::DelayedStream( +MergeTreeRangeReader::DelayedStream::DelayedStream( size_t from_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader) : current_mark(from_mark), current_offset(0), num_delayed_rows(0) , index_granularity(index_granularity), merge_tree_reader(merge_tree_reader) @@ -18,13 +18,12 @@ MergeTreePrewhereRangeReader::DelayedStream::DelayedStream( { } -size_t MergeTreePrewhereRangeReader::DelayedStream::position() const +size_t MergeTreeRangeReader::DelayedStream::position() const { return current_mark * index_granularity + current_offset + num_delayed_rows; } - -size_t MergeTreePrewhereRangeReader::DelayedStream::readRows(Block & block, size_t num_rows) +size_t MergeTreeRangeReader::DelayedStream::readRows(Block & block, size_t num_rows) { if (num_rows) { @@ -44,7 +43,7 @@ size_t MergeTreePrewhereRangeReader::DelayedStream::readRows(Block & block, size return 0; } -size_t MergeTreePrewhereRangeReader::DelayedStream::read(Block & block, size_t from_mark, size_t offset, size_t num_rows) +size_t MergeTreeRangeReader::DelayedStream::read(Block & block, size_t from_mark, size_t offset, size_t num_rows) { if (position() == from_mark * index_granularity + offset) { @@ -64,7 +63,7 @@ size_t MergeTreePrewhereRangeReader::DelayedStream::read(Block & block, size_t f } } -size_t MergeTreePrewhereRangeReader::DelayedStream::finalize(Block & block) +size_t MergeTreeRangeReader::DelayedStream::finalize(Block & block) { if (current_offset && !continue_reading) { @@ -86,27 +85,28 @@ size_t MergeTreePrewhereRangeReader::DelayedStream::finalize(Block & block) return readRows(block, rows_to_read); } -MergeTreePrewhereRangeReader::Stream::Stream(size_t from_mark, size_t to_mark, size_t index_granularity, - MergeTreeReader * merge_tree_reader) + +MergeTreeRangeReader::Stream::Stream( + size_t from_mark, size_t to_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader) : current_mark(from_mark), offset_after_current_mark(0) , index_granularity(index_granularity), last_mark(to_mark) , stream(from_mark, index_granularity, merge_tree_reader) { } -void MergeTreePrewhereRangeReader::Stream::checkNotFinished() const +void MergeTreeRangeReader::Stream::checkNotFinished() const { if (isFinished()) throw Exception("Cannot read out of marks range.", ErrorCodes::LOGICAL_ERROR); } -void MergeTreePrewhereRangeReader::Stream::checkEnoughSpaceInCurrentGranula(size_t num_rows) const +void MergeTreeRangeReader::Stream::checkEnoughSpaceInCurrentGranule(size_t num_rows) const { if (num_rows + offset_after_current_mark > index_granularity) throw Exception("Cannot read from granule more than index_granularity.", ErrorCodes::LOGICAL_ERROR); } -size_t MergeTreePrewhereRangeReader::Stream::readRows(Block & block, size_t num_rows) +size_t MergeTreeRangeReader::Stream::readRows(Block & block, size_t num_rows) { size_t rows_read = stream.read(block, current_mark, offset_after_current_mark, num_rows); @@ -116,10 +116,9 @@ size_t MergeTreePrewhereRangeReader::Stream::readRows(Block & block, size_t num_ return rows_read; } -size_t MergeTreePrewhereRangeReader::Stream::read(Block & block, size_t num_rows, - bool skip_remaining_rows_in_current_granule) +size_t MergeTreeRangeReader::Stream::read(Block & block, size_t num_rows, bool skip_remaining_rows_in_current_granule) { - checkEnoughSpaceInCurrentGranula(num_rows); + checkEnoughSpaceInCurrentGranule(num_rows); if (num_rows) { @@ -153,12 +152,12 @@ size_t MergeTreePrewhereRangeReader::Stream::read(Block & block, size_t num_rows } } -void MergeTreePrewhereRangeReader::Stream::skip(size_t num_rows) +void MergeTreeRangeReader::Stream::skip(size_t num_rows) { if (num_rows) { checkNotFinished(); - checkEnoughSpaceInCurrentGranula(num_rows); + checkEnoughSpaceInCurrentGranule(num_rows); offset_after_current_mark += num_rows; @@ -171,7 +170,7 @@ void MergeTreePrewhereRangeReader::Stream::skip(size_t num_rows) } } -size_t MergeTreePrewhereRangeReader::Stream::finalize(Block & block) +size_t MergeTreeRangeReader::Stream::finalize(Block & block) { size_t read_rows = stream.finalize(block); @@ -182,13 +181,13 @@ size_t MergeTreePrewhereRangeReader::Stream::finalize(Block & block) } -void MergeTreePrewhereRangeReader::ReadResult::addGranule(size_t num_rows) +void MergeTreeRangeReader::ReadResult::addGranule(size_t num_rows) { rows_per_granule.push_back(num_rows); num_read_rows += num_rows; } -void MergeTreePrewhereRangeReader::ReadResult::adjustLastGranule(size_t num_rows_to_subtract) +void MergeTreeRangeReader::ReadResult::adjustLastGranule(size_t num_rows_to_subtract) { if (rows_per_granule.empty()) throw Exception("Can't adjust last granule because no granules were added.", ErrorCodes::LOGICAL_ERROR); @@ -202,7 +201,7 @@ void MergeTreePrewhereRangeReader::ReadResult::adjustLastGranule(size_t num_rows num_read_rows -= num_rows_to_subtract; } -void MergeTreePrewhereRangeReader::ReadResult::clear() +void MergeTreeRangeReader::ReadResult::clear() { /// Need to save information about the number of granules. rows_per_granule.assign(rows_per_granule.size(), 0); @@ -211,9 +210,10 @@ void MergeTreePrewhereRangeReader::ReadResult::clear() num_added_rows = 0; num_zeros_in_filter = 0; filter = nullptr; + block.clear(); } -void MergeTreePrewhereRangeReader::ReadResult::optimize() +void MergeTreeRangeReader::ReadResult::optimize() { if (num_read_rows == 0 || !filter) return; @@ -243,8 +243,7 @@ void MergeTreePrewhereRangeReader::ReadResult::optimize() } } -void MergeTreePrewhereRangeReader::ReadResult::collapseZeroTails(const IColumn::Filter & filter, - IColumn::Filter & new_filter) +void MergeTreeRangeReader::ReadResult::collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter) { auto filter_data = filter.data(); auto new_filter_data = new_filter.data(); @@ -273,8 +272,7 @@ void MergeTreePrewhereRangeReader::ReadResult::collapseZeroTails(const IColumn:: new_filter.resize(new_filter_data - new_filter.data()); } - -size_t MergeTreePrewhereRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, const UInt8 * end) +size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, const UInt8 * end) { size_t count = 0; @@ -314,7 +312,7 @@ size_t MergeTreePrewhereRangeReader::ReadResult::numZerosInTail(const UInt8 * be return count; } -size_t MergeTreePrewhereRangeReader::ReadResult::numZerosInFilter() const +size_t MergeTreeRangeReader::ReadResult::numZerosInFilter() const { if (!filter) return 0; @@ -335,8 +333,7 @@ size_t MergeTreePrewhereRangeReader::ReadResult::numZerosInFilter() const return size - countBytesInFilter(*data); } - -void MergeTreePrewhereRangeReader::ReadResult::setFilter(ColumnPtr filter_) +void MergeTreeRangeReader::ReadResult::setFilter(ColumnPtr filter_) { if (!filter_ && filter) throw Exception("Can't remove exising filter with empty.", ErrorCodes::LOGICAL_ERROR); @@ -364,45 +361,111 @@ void MergeTreePrewhereRangeReader::ReadResult::setFilter(ColumnPtr filter_) num_zeros_in_filter = num_zeros; } -MergeTreePrewhereRangeReader::MergeTreePrewhereRangeReader( - MergeTreePrewhereRangeReader * prev_reader, MergeTreeReader * merge_tree_reader, - size_t from_mark, size_t to_mark, size_t index_granularity, - ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, - const Names * ordered_names, bool always_reorder) - : stream(from_mark, to_mark, index_granularity, merge_tree_reader) + +MergeTreeRangeReader::MergeTreeRangeReader( + MergeTreeReader * merge_tree_reader, size_t index_granularity, + MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions, + const String * prewhere_column_name, const Names * ordered_names, bool always_reorder) + : index_granularity(index_granularity), merge_tree_reader(merge_tree_reader) , prev_reader(prev_reader), prewhere_actions(std::move(prewhere_actions)) - , prewhere_column_name(prewhere_column_name), ordered_names(ordered_names), always_reorder(always_reorder) + , prewhere_column_name(prewhere_column_name), ordered_names(ordered_names) + , always_reorder(always_reorder), is_initialized(true) { } +bool MergeTreeRangeReader::isReadingFinished() const +{ + return prev_reader ? prev_reader->isReadingFinished() : stream.isFinished(); +} -MergeTreePrewhereRangeReader::ReadResult MergeTreePrewhereRangeReader::read( - Block & res, size_t max_rows) +size_t MergeTreeRangeReader::numReadRowsInCurrentGranule() const +{ + return prev_reader ? prev_reader->numReadRowsInCurrentGranule() : stream.numReadRowsInCurrentGranule(); +} +size_t MergeTreeRangeReader::numPendingRowsInCurrentGranule() const +{ + return prev_reader ? prev_reader->numPendingRowsInCurrentGranule() : stream.numPendingRowsInCurrentGranule(); +} + +size_t MergeTreeRangeReader::numPendingRows() const +{ + return prev_reader ? prev_reader->numPendingRows() : stream.numPendingRows(); +} + +bool isCurrentRangeFinished() const +{ + return prev_reader ? prev_reader->isCurrentRangeFinished() : stream.isFinished(); +} + +MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, MarkRanges & ranges) { if (max_rows == 0) throw Exception("Expected at least 1 row to read, got 0.", ErrorCodes::LOGICAL_ERROR); - if (max_rows > numPendingRows()) - throw Exception("Want to read " + toString(max_rows) + " rows, but has only " - + toString(numPendingRows()) + " pending rows.", ErrorCodes::LOGICAL_ERROR); - ReadResult read_result; if (prev_reader) - read_result = prev_reader->read(res, max_rows); + { + read_result = prev_reader->read(max_rows, ranges); + continueReadingChain(read_result); + } + else + read_result = startReadingChain(max_rows, ranges); - readRows(res, max_rows, read_result); - - if (!res) + if (!read_result.block) return read_result; - executePrewhereActionsAndFilterColumns(res, read_result); + executePrewhereActionsAndFilterColumns(read_result); return read_result; } -void MergeTreePrewhereRangeReader::readRows(Block & block, size_t max_rows, ReadResult & result) +MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t max_rows, MarkRanges & ranges) { - if (prev_reader && result.numReadRows() == 0) + ReadResult result; + + /// Stream is lazy. result.num_added_rows is the number of rows added to block which is not equal to + /// result.num_rows_read until call to stream.finalize(). Also result.num_added_rows may be less than + /// result.num_rows_read if the last granule in range also the last in part (so we have to adjust last granule). + { + size_t space_left = max_rows; + while (space_left && !stream.isFinished() && !ranges.empty()) + { + if (stream.isFinished()) + { + result.addRows(stream.finalize(result.block)); + ranges.pop_back(); + stream = Stream(ranges.back().begin, ranges.back().end, index_granularity, merge_tree_reader); + result.addRange(ranges.back()); + ranges.pop_back(); + } + + auto rows_to_read = std::min(space_left, stream.numPendingRowsInCurrentGranule()); + bool last = rows_to_read == space_left; + result.addRows(stream.read(result.block, rows_to_read, !last)); + result.addGranule(rows_to_read); + space_left -= rows_to_read; + } + } + + result.addRows(stream.finalize(result.block)); + + auto last_granule = result.rowsPerGranule().back(); + auto added_rows =result.getNumAddedRows(); + + if (max_rows - last_granule > added_rows) + throw Exception("RangeReader expected reading of at least " + toString(max_rows - last_granule) + + " rows, but only " + toString(added_rows) + " was read.", ErrorCodes::LOGICAL_ERROR); + + /// Last granule may be incomplete. + size_t adjustment = max_rows - added_rows; + result.adjustLastGranule(adjustment); + + return result; +} + +void MergeTreeRangeReader::continueReadingChain(ReadResult & result) +{ + if (result.numReadRows() == 0) { /// If zero rows were read on prev step, than there is no more rows to read. /// Last granule may have less rows than index_granularity, so finish reading manually. @@ -412,7 +475,6 @@ void MergeTreePrewhereRangeReader::readRows(Block & block, size_t max_rows, Read size_t rows_to_skip_in_last_granule = 0; - if (!result.rowsPerGranule().empty()) { size_t rows_in_last_granule = result.rowsPerGranule().back(); result.optimize(); @@ -425,83 +487,58 @@ void MergeTreePrewhereRangeReader::readRows(Block & block, size_t max_rows, Read } } - if (result.rowsPerGranule().empty()) + auto & rows_per_granule = result.rowsPerGranule(); + auto & started_ranges = result.startedRanges(); + + size_t added_rows = 0; + size_t next_range_to_start = 0; + + auto size = rows_per_granule.size(); + for (auto i : ext::range(0, size)) { - /// Stream is lazy. result.num_added_rows is the number of rows added to block which is not equal to - /// result.num_rows_read until call to stream.finalize(). Also result.num_added_rows may be less than - /// result.num_rows_read if the last granule in range also the last in part (so we have to adjust last granule). + if (next_range_to_start < started_ranges.size() + && i == started_ranges[next_range_to_start].num_granules_read_before_start) { - size_t space_left = max_rows; - while (space_left && !stream.isFinished()) - { - auto rows_to_read = std::min(space_left, stream.numPendingRowsInCurrentGranule()); - bool last = rows_to_read == space_left; - result.addRows(stream.read(block, rows_to_read, !last)); - result.addGranule(rows_to_read); - space_left -= rows_to_read; - } + added_rows += stream.finalize(result.block); + auto & range = started_ranges[next_range_to_start].range; + stream = Stream(range.begin, range.end, index_granularity, merge_tree_reader); } - stream.skip(rows_to_skip_in_last_granule); - result.addRows(stream.finalize(block)); - - auto last_granule = result.rowsPerGranule().back(); - - auto added_rows =result.getNumAddedRows(); - - if (max_rows - last_granule > added_rows) - throw Exception("RangeReader expected reading of at least " + toString(max_rows - last_granule) + - " rows, but only " + toString(added_rows) + " was read.", ErrorCodes::LOGICAL_ERROR); - - /// Last granule may be incomplete. - size_t adjustment = max_rows - added_rows; - result.adjustLastGranule(adjustment); - + bool last = i + 1 == size; + added_rows += stream.read(result.block, rows_per_granule[i], !last); } - else - { - size_t added_rows = 0; - auto & rows_per_granule = result.rowsPerGranule(); - auto size = rows_per_granule.size(); - for (auto i : ext::range(0, size)) - { - bool last = i + 1 == size; - added_rows += stream.read(block, rows_per_granule[i], !last); - } + stream.skip(rows_to_skip_in_last_granule); + added_rows += stream.finalize(result.block); - stream.skip(rows_to_skip_in_last_granule); - added_rows += stream.finalize(block); - - /// added_rows may be zero if all columns were read in prewhere and it's ok. - if (added_rows && added_rows != result.numReadRows()) - throw Exception("RangeReader read " + toString(added_rows) + " rows, but " - + toString(result.numReadRows()) + " expected.", ErrorCodes::LOGICAL_ERROR); - } + /// added_rows may be zero if all columns were read in prewhere and it's ok. + if (added_rows && added_rows != result.numReadRows()) + throw Exception("RangeReader read " + toString(added_rows) + " rows, but " + + toString(result.numReadRows()) + " expected.", ErrorCodes::LOGICAL_ERROR); } -void MergeTreePrewhereRangeReader::executePrewhereActionsAndFilterColumns(Block & block, ReadResult & result) +void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { const auto & columns = stream.reader()->getColumns(); - auto filterColumns = [&block, &columns](const IColumn::Filter & filter) + auto filterColumns = [&result, &columns](const IColumn::Filter & filter) { for (const auto & column : columns) { - if (block.has(column.name)) + if (result.block.has(column.name)) { - auto & column_with_type_and_name = block.getByName(column.name); + auto & column_with_type_and_name = result.block.getByName(column.name); column_with_type_and_name.column = std::move(column_with_type_and_name.column)->filter(filter, -1); } } }; - auto filterBlock = [&block](const IColumn::Filter & filter) + auto filterBlock = [&result](const IColumn::Filter & filter) { - for (const auto i : ext::range(0, block.columns())) + for (const auto i : ext::range(0, result.block.columns())) { - auto & col = block.safeGetByPosition(i); + auto & col = result.block.safeGetByPosition(i); if (col.column && col.column->size() == filter.size()) col.column = std::move(col.column)->filter(filter, -1); @@ -521,56 +558,18 @@ void MergeTreePrewhereRangeReader::executePrewhereActionsAndFilterColumns(Block } } - if (!columns.empty()) - { - if (columns.size() == block.columns()) - { - stream.reader()->fillMissingColumns(block, *ordered_names, always_reorder); - - if (prewhere_actions) - prewhere_actions->execute(block); - } - else - { - /// Columns in block may have different size here. Create temporary block which has only read columns. - Block tmp_block; - for (const auto & column : columns) - { - if (block.has(column.name)) - { - auto & column_with_type_and_name = block.getByName(column.name); - tmp_block.insert(column_with_type_and_name); - column_with_type_and_name.column = nullptr; - } - } - - if (tmp_block) - stream.reader()->fillMissingColumns(tmp_block, *ordered_names, always_reorder); - - if (prewhere_actions) - prewhere_actions->execute(tmp_block); - - for (auto col_num : ext::range(0, block.columns())) - { - auto & column = block.getByPosition(col_num); - if (!tmp_block.has(column.name)) - tmp_block.insert(std::move(column)); - } - - std::swap(block, tmp_block); - } - } + stream.reader()->fillMissingColumns(result.block, *ordered_names, always_reorder); ColumnPtr filter; if (prewhere_actions) { - auto & prewhere_column = block.getByName(*prewhere_column_name); + prewhere_actions->execute(result.block); + auto & prewhere_column = result.block.getByName(*prewhere_column_name); ConstantFilterDescription constant_filter_description(*prewhere_column.column); if (constant_filter_description.always_false) { result.clear(); - block.clear(); return; } else if (!constant_filter_description.always_true) @@ -580,14 +579,14 @@ void MergeTreePrewhereRangeReader::executePrewhereActionsAndFilterColumns(Block filterBlock(*filter_and_holder.data); } - prewhere_column.column = prewhere_column.type->createColumnConst(block.rows(), UInt64(1)); + prewhere_column.column = prewhere_column.type->createColumnConst(result.block.rows(), UInt64(1)); } if (filter && result.getFilter()) { /// TODO: implement for prewhere chain. /// In order to do it we need combine filter and result.filter, where filter filters only '1' in result.filter. - throw Exception("MergeTreePrewhereRangeReader chain with several prewhere actions in not implemented.", + throw Exception("MergeTreeRangeReader chain with several prewhere actions in not implemented.", ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 1d8601fe573..867888303da 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -10,15 +10,14 @@ class MergeTreeReader; /// MergeTreeReader iterator which allows sequential reading for arbitrary number of rows between pairs of marks in the same part. /// Stores reading state, which can be inside granule. Can skip rows in current granule and start reading from next mark. /// Used generally for reading number of rows less than index granularity to decrease cache misses for fat blocks. -class MergeTreePrewhereRangeReader +class MergeTreeRangeReader { public: - MergeTreePrewhereRangeReader(MergeTreePrewhereRangeReader * prev_reader, MergeTreeReader * merge_tree_reader, - size_t from_mark, size_t to_mark, size_t index_granularity, - ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, - const Names * ordered_names, bool always_reorder); + MergeTreeRangeReader(MergeTreeReader * merge_tree_reader, size_t index_granularity, + MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions, + const String * prewhere_column_name, const Names * ordered_names, bool always_reorder); - MergeTreePrewhereRangeReader() : is_initialized(false) { } + MergeTreeRangeReader() = default; bool isReadingFinished() const { return prev_reader ? prev_reader->isReadingFinished() : stream.isFinished(); } @@ -26,13 +25,14 @@ public: size_t numPendingRowsInCurrentGranule() const { return prev_reader ? prev_reader->numPendingRowsInCurrentGranule() : stream.numPendingRowsInCurrentGranule(); } size_t numPendingRows() const { return prev_reader ? prev_reader->numPendingRows() : stream.numPendingRows(); } - operator bool() const { return is_initialized; } - void reset() { is_initialized = false; } + bool isCurrentRangeFinished() const { return prev_reader ? prev_reader->isCurrentRangeFinished() : stream.isFinished(); } + + bool isInitialized() const { return is_initialized; } class DelayedStream { public: - DelayedStream() {} + DelayedStream() = default; DelayedStream(size_t from_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader); /// Returns the number of rows added to block. @@ -46,14 +46,14 @@ public: MergeTreeReader * reader() const { return merge_tree_reader; } private: - size_t current_mark; - size_t current_offset; - size_t num_delayed_rows; + size_t current_mark = 0; + size_t current_offset = 0; + size_t num_delayed_rows = 0; - size_t index_granularity; - MergeTreeReader * merge_tree_reader; - bool continue_reading; - bool is_finished; + size_t index_granularity = 0; + MergeTreeReader * merge_tree_reader = nullptr; + bool continue_reading = false; + bool is_finished = true; size_t position() const; size_t readRows(Block & block, size_t num_rows); @@ -63,7 +63,7 @@ public: { public: - Stream() {} + Stream() = default; Stream(size_t from_mark, size_t to_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader); /// Returns the n @@ -82,17 +82,17 @@ public: MergeTreeReader * reader() const { return stream.reader(); } private: - size_t current_mark; + size_t current_mark = 0; /// Invariant: offset_after_current_mark + skipped_rows_after_offset < index_granularity - size_t offset_after_current_mark; + size_t offset_after_current_mark = 0; - size_t index_granularity; - size_t last_mark; + size_t index_granularity = 0; + size_t last_mark = 0; DelayedStream stream; void checkNotFinished() const; - void checkEnoughSpaceInCurrentGranula(size_t num_rows) const; + void checkEnoughSpaceInCurrentGranule(size_t num_rows) const; size_t readRows(Block & block, size_t num_rows); }; @@ -100,7 +100,15 @@ public: class ReadResult { public: + struct RangeInfo + { + size_t num_granules_read_before_start; + MarkRange range; + }; + + const std::vector & startedRanges() const { return started_ranges; } const std::vector & rowsPerGranule() const { return rows_per_granule; } + /// The number of rows were read at LAST iteration in chain. <= num_added_rows + num_filtered_rows. size_t numReadRows() const { return num_read_rows; } /// The number of rows were added to block as a result of reading chain. @@ -113,6 +121,7 @@ public: void addGranule(size_t num_rows); void adjustLastGranule(size_t num_rows_to_subtract); void addRows(size_t rows) { num_added_rows += rows; } + void addRange(const MarkRange & range) { started_ranges.emplace_back(rows_per_granule.size(), range); } /// Set filter or replace old one. Filter must have more zeroes than previous. void setFilter(ColumnPtr filter_); @@ -121,7 +130,10 @@ public: /// Remove all rows from granules. void clear(); + Block block; + private: + std::vector started_ranges; /// The number of rows read from each granule. std::vector rows_per_granule; /// Sum(rows_per_granule) @@ -140,22 +152,27 @@ public: static size_t numZerosInTail(const UInt8 * begin, const UInt8 * end); }; - ReadResult read(Block & res, size_t max_rows); + ReadResult read(size_t max_rows, MarkRanges & ranges); private: - void readRows(Block & block, size_t max_rows, ReadResult & result); - void executePrewhereActionsAndFilterColumns(Block & block, ReadResult & result); + ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); + void continueReadingChain(ReadResult & result); + void executePrewhereActionsAndFilterColumns(ReadResult & result); + + size_t index_granularity = 0; + MergeTreeReader * merge_tree_reader = nullptr; + + MergeTreeRangeReader * prev_reader = nullptr; /// If not nullptr, read from prev_reader firstly. + + ExpressionActionsPtr prewhere_actions = nullptr; /// If not nullptr, calculate filter. + const String * prewhere_column_name = nullptr; + const Names * ordered_names = nullptr; Stream stream; - MergeTreePrewhereRangeReader * prev_reader; /// If not nullptr, read from prev_reader firstly. - ExpressionActionsPtr prewhere_actions; /// If not nullptr, calculate filter. - const String * prewhere_column_name; - const Names * ordered_names; - bool always_reorder; - - bool is_initialized = true; + bool always_reorder = true; + bool is_initialized = false; }; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 73473192ce8..4f5252ccdc3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -66,17 +66,6 @@ const MergeTreeReader::ValueSizeMap & MergeTreeReader::getAvgValueSizeHints() co } -MergeTreePrewhereRangeReader MergeTreeReader::readRange( - size_t from_mark, size_t to_mark, MergeTreePrewhereRangeReader * prev_reader, - ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, - const Names * ordered_names, bool always_reorder) -{ - return MergeTreePrewhereRangeReader( - prev_reader, this, from_mark, to_mark, storage.index_granularity, - prewhere_actions, prewhere_column_name, ordered_names, always_reorder); -} - - size_t MergeTreeReader::readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res) { size_t read_rows = 0; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index a9d384ce8ef..e13c03a1ae2 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -38,11 +38,6 @@ public: const ValueSizeMap & getAvgValueSizeHints() const; - /// Create MergeTreeRangeReader iterator, which allows reading arbitrary number of rows from range. - MergeTreePrewhereRangeReader readRange(size_t from_mark, size_t to_mark, MergeTreePrewhereRangeReader * prev_reader, - ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, - const Names * ordered_names, bool always_reorder); - /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// If at least one column was added, reorders all columns in the block according to ordered_names. @@ -121,7 +116,7 @@ private: /// If continue_reading is true, continue reading from last state, otherwise seek to from_mark size_t readRows(size_t from_mark, bool continue_reading, size_t max_rows_to_read, Block & res); - friend class MergeTreePrewhereRangeReader::DelayedStream; + friend class MergeTreeRangeReader::DelayedStream; }; } From af6895ee6f1cd28b49bf30c8b4a2592994718e05 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 14:50:33 +0300 Subject: [PATCH 03/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- .../src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeRangeReader.h | 13 ++++++------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 974db6131ee..4d30b7522f8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -392,7 +392,7 @@ size_t MergeTreeRangeReader::numPendingRows() const return prev_reader ? prev_reader->numPendingRows() : stream.numPendingRows(); } -bool isCurrentRangeFinished() const +bool MergeTreeRangeReader::isCurrentRangeFinished() const { return prev_reader ? prev_reader->isCurrentRangeFinished() : stream.isFinished(); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 867888303da..2e0ce779bab 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -19,14 +19,13 @@ public: MergeTreeRangeReader() = default; - bool isReadingFinished() const { return prev_reader ? prev_reader->isReadingFinished() : stream.isFinished(); } + bool isReadingFinished() const; - size_t numReadRowsInCurrentGranule() const { return prev_reader ? prev_reader->numReadRowsInCurrentGranule() : stream.numReadRowsInCurrentGranule(); } - size_t numPendingRowsInCurrentGranule() const { return prev_reader ? prev_reader->numPendingRowsInCurrentGranule() : stream.numPendingRowsInCurrentGranule(); } - size_t numPendingRows() const { return prev_reader ? prev_reader->numPendingRows() : stream.numPendingRows(); } - - bool isCurrentRangeFinished() const { return prev_reader ? prev_reader->isCurrentRangeFinished() : stream.isFinished(); } + size_t numReadRowsInCurrentGranule() const; + size_t numPendingRowsInCurrentGranule() const; + size_t numPendingRows() const; + bool isCurrentRangeFinished() const; bool isInitialized() const { return is_initialized; } class DelayedStream @@ -121,7 +120,7 @@ public: void addGranule(size_t num_rows); void adjustLastGranule(size_t num_rows_to_subtract); void addRows(size_t rows) { num_added_rows += rows; } - void addRange(const MarkRange & range) { started_ranges.emplace_back(rows_per_granule.size(), range); } + void addRange(const MarkRange & range) { started_ranges.emplace_back({ rows_per_granule.size(), range }); } /// Set filter or replace old one. Filter must have more zeroes than previous. void setFilter(ColumnPtr filter_); From 58b6b7570fe40be277ee696f75d2b338c48200c9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 14:54:32 +0300 Subject: [PATCH 04/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- dbms/src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 2e0ce779bab..bcb78b8718c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -120,7 +120,7 @@ public: void addGranule(size_t num_rows); void adjustLastGranule(size_t num_rows_to_subtract); void addRows(size_t rows) { num_added_rows += rows; } - void addRange(const MarkRange & range) { started_ranges.emplace_back({ rows_per_granule.size(), range }); } + void addRange(const MarkRange & range) { started_ranges.push_back({rows_per_granule.size(), range}); } /// Set filter or replace old one. Filter must have more zeroes than previous. void setFilter(ColumnPtr filter_); From 77a3989558fa7032b874b61561dc0f59d3316276 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 15:02:25 +0300 Subject: [PATCH 05/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp | 2 +- dbms/src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 4d30b7522f8..7e28f20891d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -428,7 +428,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t /// result.num_rows_read if the last granule in range also the last in part (so we have to adjust last granule). { size_t space_left = max_rows; - while (space_left && !stream.isFinished() && !ranges.empty()) + while (space_left && (!stream.isFinished() || !ranges.empty())) { if (stream.isFinished()) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index bcb78b8718c..59fb4c3deb8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -114,7 +114,7 @@ public: size_t getNumAddedRows() const { return num_added_rows; } /// The number of filtered rows at all steps in reading chain. size_t getNumFilteredRows() const { return num_filtered_rows; } - /// Filter you need to allply to newly-read columns in order to add them to block. + /// Filter you need to apply to newly-read columns in order to add them to block. const ColumnPtr & getFilter() const { return filter; } void addGranule(size_t num_rows); From d2d0d570d83966201db69f074ddd73a34e365c49 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 15:09:32 +0300 Subject: [PATCH 06/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 7e28f20891d..1a7a3e2b4de 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -433,7 +433,6 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t if (stream.isFinished()) { result.addRows(stream.finalize(result.block)); - ranges.pop_back(); stream = Stream(ranges.back().begin, ranges.back().end, index_granularity, merge_tree_reader); result.addRange(ranges.back()); ranges.pop_back(); From d8b5f14dd727a6dd6c274db27cf14ebe4b23d9e4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 15:26:09 +0300 Subject: [PATCH 07/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp | 9 +++++---- dbms/src/Storages/MergeTree/MergeTreeRangeReader.h | 4 ---- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 1a7a3e2b4de..83cbefed03a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -519,7 +519,7 @@ void MergeTreeRangeReader::continueReadingChain(ReadResult & result) void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - const auto & columns = stream.reader()->getColumns(); + const auto & columns = merge_tree_reader->getColumns(); auto filterColumns = [&result, &columns](const IColumn::Filter & filter) { @@ -528,7 +528,8 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (result.block.has(column.name)) { auto & column_with_type_and_name = result.block.getByName(column.name); - column_with_type_and_name.column = std::move(column_with_type_and_name.column)->filter(filter, -1); + if (column_with_type_and_name.column) + column_with_type_and_name.column = column_with_type_and_name.column->filter(filter, -1); } } }; @@ -540,7 +541,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r auto & col = result.block.safeGetByPosition(i); if (col.column && col.column->size() == filter.size()) - col.column = std::move(col.column)->filter(filter, -1); + col.column = col.column->filter(filter, -1); } }; @@ -557,7 +558,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r } } - stream.reader()->fillMissingColumns(result.block, *ordered_names, always_reorder); + merge_tree_reader->fillMissingColumns(result.block, *ordered_names, always_reorder); ColumnPtr filter; if (prewhere_actions) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 59fb4c3deb8..a2a373c0cca 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -42,8 +42,6 @@ public: bool isFinished() const { return is_finished; } - MergeTreeReader * reader() const { return merge_tree_reader; } - private: size_t current_mark = 0; size_t current_offset = 0; @@ -78,8 +76,6 @@ public: size_t numRendingGranules() const { return last_mark - current_mark; } size_t numPendingRows() const { return numRendingGranules() * index_granularity - offset_after_current_mark; } - MergeTreeReader * reader() const { return stream.reader(); } - private: size_t current_mark = 0; /// Invariant: offset_after_current_mark + skipped_rows_after_offset < index_granularity From 3063c35429af29904b3a926d2494a150f64af0f6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 16:23:56 +0300 Subject: [PATCH 08/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 1cb567a0caf..31d2a083210 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -125,7 +125,8 @@ Block MergeTreeBaseBlockInputStream::readFromPart() &prewhere_column_name, &task->ordered_names, task->should_reorder); task->range_reader = MergeTreeRangeReader( - reader.get(), index_granularity, &task->pre_range_reader, nullptr, nullptr, nullptr, true); + reader.get(), index_granularity, &task->pre_range_reader, nullptr, + nullptr, &task->ordered_names, true); } else { From dcd99860fe720651f16b0f9abc6a94500f8775fd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 16:30:47 +0300 Subject: [PATCH 09/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- .../Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp | 4 ++-- dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp | 7 ++++--- dbms/src/Storages/MergeTree/MergeTreeRangeReader.h | 4 ++-- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 31d2a083210..8bd4b469593 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -147,8 +147,8 @@ Block MergeTreeBaseBlockInputStream::readFromPart() if (task->size_predictor) { task->size_predictor->updateFilteredRowsRation( - read_result.getNumAddedRows() + read_result.getNumFilteredRows(), - read_result.getNumFilteredRows()); + read_result.numAddedRows() + read_result.numFilteredRows(), + read_result.numFilteredRows()); if (read_result.block) task->size_predictor->update(read_result.block); diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 83cbefed03a..70628f2e394 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -449,10 +449,11 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t result.addRows(stream.finalize(result.block)); auto last_granule = result.rowsPerGranule().back(); - auto added_rows =result.getNumAddedRows(); + auto added_rows = result.numAddedRows(); + auto num_read_rows = result.numReadRows(); - if (max_rows - last_granule > added_rows) - throw Exception("RangeReader expected reading of at least " + toString(max_rows - last_granule) + + if (num_read_rows - last_granule > added_rows) + throw Exception("RangeReader expected reading of at least " + toString(num_read_rows - last_granule) + " rows, but only " + toString(added_rows) + " was read.", ErrorCodes::LOGICAL_ERROR); /// Last granule may be incomplete. diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index a2a373c0cca..055b85ba78d 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -107,9 +107,9 @@ public: /// The number of rows were read at LAST iteration in chain. <= num_added_rows + num_filtered_rows. size_t numReadRows() const { return num_read_rows; } /// The number of rows were added to block as a result of reading chain. - size_t getNumAddedRows() const { return num_added_rows; } + size_t numAddedRows() const { return num_added_rows; } /// The number of filtered rows at all steps in reading chain. - size_t getNumFilteredRows() const { return num_filtered_rows; } + size_t numFilteredRows() const { return num_filtered_rows; } /// Filter you need to apply to newly-read columns in order to add them to block. const ColumnPtr & getFilter() const { return filter; } From 2bf1931d86e9be41286d194a81b5050c1837b2df Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 16:37:04 +0300 Subject: [PATCH 10/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- .../Storages/MergeTree/MergeTreeRangeReader.cpp | 15 ++++----------- .../src/Storages/MergeTree/MergeTreeRangeReader.h | 2 +- 2 files changed, 5 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 70628f2e394..01b0b58dd72 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -187,8 +187,10 @@ void MergeTreeRangeReader::ReadResult::addGranule(size_t num_rows) num_read_rows += num_rows; } -void MergeTreeRangeReader::ReadResult::adjustLastGranule(size_t num_rows_to_subtract) +void MergeTreeRangeReader::ReadResult::adjustLastGranule() { + size_t num_rows_to_subtract = num_read_rows - num_added_rows; + if (rows_per_granule.empty()) throw Exception("Can't adjust last granule because no granules were added.", ErrorCodes::LOGICAL_ERROR); @@ -448,17 +450,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t result.addRows(stream.finalize(result.block)); - auto last_granule = result.rowsPerGranule().back(); - auto added_rows = result.numAddedRows(); - auto num_read_rows = result.numReadRows(); - - if (num_read_rows - last_granule > added_rows) - throw Exception("RangeReader expected reading of at least " + toString(num_read_rows - last_granule) + - " rows, but only " + toString(added_rows) + " was read.", ErrorCodes::LOGICAL_ERROR); - /// Last granule may be incomplete. - size_t adjustment = max_rows - added_rows; - result.adjustLastGranule(adjustment); + result.adjustLastGranule(); return result; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 055b85ba78d..58590a08630 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -114,7 +114,7 @@ public: const ColumnPtr & getFilter() const { return filter; } void addGranule(size_t num_rows); - void adjustLastGranule(size_t num_rows_to_subtract); + void adjustLastGranule(); void addRows(size_t rows) { num_added_rows += rows; } void addRange(const MarkRange & range) { started_ranges.push_back({rows_per_granule.size(), range}); } From 36fe36d00359ca4d4bb91b72651d70b2bb09e810 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 16:59:19 +0300 Subject: [PATCH 11/70] updated MergeTreePrewhereRangeReader; renamed MergeTreePrewhereRangeReader to MergeTreeRangeReader --- dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 8bd4b469593..c2687a4303a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -142,6 +142,10 @@ Block MergeTreeBaseBlockInputStream::readFromPart() auto read_result = task->range_reader.read(rows_to_read, task->mark_ranges); + /// All rows were filtered. Repeat. + if (read_result.block.rows() == 0) + read_result.block.clear(); + progressImpl({ read_result.block.rows(), read_result.block.bytes() }); if (task->size_predictor) From ba81835e5864a2ccc6a843bbfcac70bdd3b12855 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 20 Feb 2018 17:26:22 +0300 Subject: [PATCH 12/70] fixed progress stats --- .../Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp | 8 ++++---- dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp | 4 ++++ dbms/src/Storages/MergeTree/MergeTreeRangeReader.h | 6 ++++++ 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index c2687a4303a..96216d45325 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -146,13 +146,13 @@ Block MergeTreeBaseBlockInputStream::readFromPart() if (read_result.block.rows() == 0) read_result.block.clear(); - progressImpl({ read_result.block.rows(), read_result.block.bytes() }); + size_t rows_read = read_result.numAddedRows() + read_result.numFilteredRows(); + + progressImpl({ rows_read, read_result.numBytesRead() }); if (task->size_predictor) { - task->size_predictor->updateFilteredRowsRation( - read_result.numAddedRows() + read_result.numFilteredRows(), - read_result.numFilteredRows()); + task->size_predictor->updateFilteredRowsRation(rows_read, read_result.numFilteredRows()); if (read_result.block) task->size_predictor->update(read_result.block); diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 01b0b58dd72..6656e480c98 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -405,10 +405,12 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar throw Exception("Expected at least 1 row to read, got 0.", ErrorCodes::LOGICAL_ERROR); ReadResult read_result; + size_t prev_bytes = 0; if (prev_reader) { read_result = prev_reader->read(max_rows, ranges); + prev_bytes = read_result.block.bytes(); continueReadingChain(read_result); } else @@ -417,6 +419,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar if (!read_result.block) return read_result; + read_result.addNumBytesRead(read_result.block.bytes() - prev_bytes); + executePrewhereActionsAndFilterColumns(read_result); return read_result; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 58590a08630..301339ee3c8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -110,6 +110,8 @@ public: size_t numAddedRows() const { return num_added_rows; } /// The number of filtered rows at all steps in reading chain. size_t numFilteredRows() const { return num_filtered_rows; } + /// The number of bytes read from disk. + size_t numBytesRead() const { return num_bytes_read; } /// Filter you need to apply to newly-read columns in order to add them to block. const ColumnPtr & getFilter() const { return filter; } @@ -125,6 +127,8 @@ public: /// Remove all rows from granules. void clear(); + void addNumBytesRead(size_t count) { num_bytes_read += count; } + Block block; private: @@ -139,6 +143,8 @@ public: size_t num_filtered_rows = 0; /// Zero if filter is nullptr. size_t num_zeros_in_filter = 0; + /// Without any filtration. + size_t num_bytes_read = 0; /// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= total_rows_read. ColumnPtr filter; From 39f16b66b5ca058c0b83da00aa3356d9607c4e85 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Feb 2018 14:31:15 +0300 Subject: [PATCH 13/70] fix continueReadingChain from several mark ranges --- .../MergeTree/MergeTreeRangeReader.cpp | 40 ++++++++++--------- .../Storages/MergeTree/MergeTreeRangeReader.h | 2 +- 2 files changed, 22 insertions(+), 20 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 6656e480c98..729976355e0 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -352,15 +352,23 @@ void MergeTreeRangeReader::ReadResult::setFilter(ColumnPtr filter_) + " has size " + toString(filter->size()) + ".", ErrorCodes::LOGICAL_ERROR); filter = std::move(filter_); + size_t num_zeros = numZerosInFilter(); if (num_zeros < num_zeros_in_filter) throw Exception("New filter has less zeros than previous.", ErrorCodes::LOGICAL_ERROR); - size_t added_zeros = num_zeros - num_zeros_in_filter; - num_added_rows -= added_zeros; - num_filtered_rows += added_zeros; - num_zeros_in_filter = num_zeros; + if (num_zeros == filter->size()) + clear(); + else if (num_zeros == 0) + filter = nullptr; + else + { + size_t added_zeros = num_zeros - num_zeros_in_filter; + num_added_rows -= added_zeros; + num_filtered_rows += added_zeros; + num_zeros_in_filter = num_zeros; + } } @@ -498,6 +506,7 @@ void MergeTreeRangeReader::continueReadingChain(ReadResult & result) { added_rows += stream.finalize(result.block); auto & range = started_ranges[next_range_to_start].range; + ++next_range_to_start; stream = Stream(range.begin, range.end, index_granularity, merge_tree_reader); } @@ -563,20 +572,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r { prewhere_actions->execute(result.block); auto & prewhere_column = result.block.getByName(*prewhere_column_name); - - ConstantFilterDescription constant_filter_description(*prewhere_column.column); - if (constant_filter_description.always_false) - { - result.clear(); - return; - } - else if (!constant_filter_description.always_true) - { - filter = std::move(prewhere_column.column); - FilterDescription filter_and_holder(*filter); - filterBlock(*filter_and_holder.data); - } - + filter = std::move(prewhere_column.column); prewhere_column.column = prewhere_column.type->createColumnConst(result.block.rows(), UInt64(1)); } @@ -588,8 +584,14 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r ErrorCodes::LOGICAL_ERROR); } + result.setFilter(filter); + + filter = result.getFilter(); if (filter) - result.setFilter(filter); + { + FilterDescription filter_and_holder(*filter); + filterBlock(*filter_and_holder.data); + } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 301339ee3c8..3c10256e90b 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -145,7 +145,7 @@ public: size_t num_zeros_in_filter = 0; /// Without any filtration. size_t num_bytes_read = 0; - /// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= total_rows_read. + /// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= num_read_rows. ColumnPtr filter; void collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter); From 733156438accbbb13f783eebd08aaceab002467b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Feb 2018 14:35:06 +0300 Subject: [PATCH 14/70] fix continueReadingChain from several mark ranges --- dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 729976355e0..5ccc1240119 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -584,7 +584,8 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r ErrorCodes::LOGICAL_ERROR); } - result.setFilter(filter); + if (filter) + result.setFilter(filter); filter = result.getFilter(); if (filter) From 7851e71a2a82f387d099f35c2bcc06180b3a0c39 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 22 Feb 2018 15:43:57 +0300 Subject: [PATCH 15/70] fix continueReadingChain from several mark ranges --- .../MergeTreeBaseBlockInputStream.cpp | 10 +- .../MergeTree/MergeTreeRangeReader.cpp | 234 ++++++++++-------- .../Storages/MergeTree/MergeTreeRangeReader.h | 22 +- .../Storages/MergeTree/MergeTreeReader.cpp | 52 ++-- dbms/src/Storages/MergeTree/MergeTreeReader.h | 6 +- 5 files changed, 195 insertions(+), 129 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 3db175451d7..21ca7a2c9f8 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -71,8 +71,6 @@ Block MergeTreeBaseBlockInputStream::readImpl() Block MergeTreeBaseBlockInputStream::readFromPart() { - Block res; - if (task->size_predictor) task->size_predictor->startBlock(); @@ -159,18 +157,18 @@ Block MergeTreeBaseBlockInputStream::readFromPart() task->size_predictor->update(read_result.block); } - if (!prewhere_column_name.empty() && res.has(prewhere_column_name)) + if (!prewhere_column_name.empty() && read_result.block.has(prewhere_column_name)) { if (task->remove_prewhere_column) - res.erase(prewhere_column_name); + read_result.block.erase(prewhere_column_name); else { - auto & column =res.getByName(prewhere_column_name); + auto & column = read_result.block.getByName(prewhere_column_name); column.column = column.column->convertToFullColumnIfConst(); } } - res.checkNumberOfRows(); + read_result.block.checkNumberOfRows(); return read_result.block; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 5ccc1240119..43313998251 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -206,13 +206,13 @@ void MergeTreeRangeReader::ReadResult::adjustLastGranule() void MergeTreeRangeReader::ReadResult::clear() { /// Need to save information about the number of granules. + num_rows_to_skip_in_last_granule += rows_per_granule.back(); rows_per_granule.assign(rows_per_granule.size(), 0); num_filtered_rows += num_read_rows - num_zeros_in_filter; num_read_rows = 0; num_added_rows = 0; num_zeros_in_filter = 0; filter = nullptr; - block.clear(); } void MergeTreeRangeReader::ReadResult::optimize() @@ -228,40 +228,63 @@ void MergeTreeRangeReader::ReadResult::optimize() filter = nullptr; else { - ColumnPtr prev_filter = std::move(filter); - FilterDescription prev_description(*prev_filter); + FilterDescription prev_description(*filter); - MutableColumnPtr new_filter_ptr = ColumnUInt8::create(prev_description.data->size()); - auto & new_filter = static_cast(*new_filter_ptr); - IColumn::Filter & new_data = new_filter.getData(); + NumRows zero_tails; + auto total_zero_rows_in_tails = countZeroTails(*prev_description.data, zero_tails); - collapseZeroTails(*prev_description.data, new_data); + /// Just a guess. If only a few rows may be skipped, it's better not to skip at all. + if (2 * total_zero_rows_in_tails > filter->size()) + { - size_t num_removed_zeroes = new_filter.size() - num_read_rows; - num_read_rows = new_filter.size(); - num_zeros_in_filter -= num_removed_zeroes; + auto new_filter = ColumnUInt8::create(prev_description.data->size() - total_zero_rows_in_tails); + IColumn::Filter & new_data = new_filter->getData(); - filter = std::move(new_filter_ptr); + size_t rows_in_last_granule = rows_per_granule.back(); + + collapseZeroTails(*prev_description.data, new_data, zero_tails); + + size_t num_removed_zeroes = new_filter->size() - num_read_rows; + num_read_rows = new_filter->size(); + num_zeros_in_filter -= num_removed_zeroes; + num_rows_to_skip_in_last_granule += rows_in_last_granule - rows_per_granule.back(); + + filter = std::move(new_filter); + } } } -void MergeTreeRangeReader::ReadResult::collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter) +size_t MergeTreeRangeReader::ReadResult::countZeroTails(const IColumn::Filter & filter, NumRows & zero_tails) const +{ + zero_tails.resize(0); + zero_tails.reserve(rows_per_granule.size()); + + auto filter_data = filter.data(); + + size_t total_zero_rows_in_tails = 0; + + for (auto rows_to_read : rows_per_granule) + { + /// Count the number of zeros at the end of filter for rows were read from current granule. + zero_tails.push_back(numZerosInTail(filter_data, filter_data + rows_to_read)); + total_zero_rows_in_tails += zero_tails.back(); + filter_data += rows_to_read; + } + + return total_zero_rows_in_tails; +} + +void MergeTreeRangeReader::ReadResult::collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter, + const NumRows & zero_tails) { auto filter_data = filter.data(); auto new_filter_data = new_filter.data(); - size_t rows_in_filter_from_prev_iteration = filter.size() - num_read_rows; - if (rows_in_filter_from_prev_iteration) + for (auto i : ext::range(0, rows_per_granule.size())) { - memcpySmallAllowReadWriteOverflow15(new_filter_data, filter_data, rows_in_filter_from_prev_iteration); - filter_data += rows_in_filter_from_prev_iteration; - new_filter_data += rows_in_filter_from_prev_iteration; - } + auto & rows_to_read = rows_per_granule[i]; + auto filtered_rows_num_at_granule_end = zero_tails[i]; - for (auto & rows_to_read : rows_per_granule) - { - /// Count the number of zeros at the end of filter for rows were read from current granule. - size_t filtered_rows_num_at_granule_end = numZerosInTail(filter_data, filter_data + rows_to_read); rows_to_read -= filtered_rows_num_at_granule_end; memcpySmallAllowReadWriteOverflow15(new_filter_data, filter_data, rows_to_read); @@ -394,12 +417,12 @@ size_t MergeTreeRangeReader::numReadRowsInCurrentGranule() const } size_t MergeTreeRangeReader::numPendingRowsInCurrentGranule() const { - return prev_reader ? prev_reader->numPendingRowsInCurrentGranule() : stream.numPendingRowsInCurrentGranule(); -} + if (prev_reader) + return prev_reader->numPendingRowsInCurrentGranule(); -size_t MergeTreeRangeReader::numPendingRows() const -{ - return prev_reader ? prev_reader->numPendingRows() : stream.numPendingRows(); + auto pending_rows = stream.numPendingRowsInCurrentGranule(); + /// If pending_rows is zero, than stream is not initialized. + return pending_rows ? pending_rows : index_granularity; } bool MergeTreeRangeReader::isCurrentRangeFinished() const @@ -419,10 +442,50 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar { read_result = prev_reader->read(max_rows, ranges); prev_bytes = read_result.block.bytes(); - continueReadingChain(read_result); + Block block = continueReadingChain(read_result); + + bool should_reorder = false; + bool should_evaluate_missing_defaults = false; + if (block) + { + /// block.rows() <= read_result.block. We must filter block before adding columns to read_result.block + + /// Fill missing columns before filtering because some arrays from Nested may have empty data. + merge_tree_reader->fillMissingColumns(block, should_reorder, should_evaluate_missing_defaults); + + const auto & filter = read_result.getFilter(); + if (filter) + filterBlock(block, filter); + + for (auto i : ext::range(0, block.columns())) + read_result.block.insert(std::move(block.getByPosition(i))); + } + + if (read_result.block) + { + if (should_evaluate_missing_defaults) + merge_tree_reader->evaluateMissingDefaults(read_result.block); + + if (should_reorder || always_reorder || block.columns()) + merge_tree_reader->reorderColumns(read_result.block, *ordered_names); + } } else + { read_result = startReadingChain(max_rows, ranges); + if (read_result.block) + { + bool should_reorder; + bool should_evaluate_missing_defaults; + merge_tree_reader->fillMissingColumns(read_result.block, should_reorder, should_evaluate_missing_defaults); + + if (should_evaluate_missing_defaults) + merge_tree_reader->evaluateMissingDefaults(read_result.block); + + if (should_reorder || always_reorder) + merge_tree_reader->reorderColumns(read_result.block, *ordered_names); + } + } if (!read_result.block) return read_result; @@ -433,6 +496,31 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar return read_result; } +void MergeTreeRangeReader::filterBlock(Block & block, const ColumnPtr & filter) const +{ + if (!filter) + return; + + FilterDescription filter_and_holder(*filter); + + auto bytes_in_filter = countBytesInFilter(*filter_and_holder.data); + if (bytes_in_filter == filter->size()) + return; + else if (bytes_in_filter == 0) + { + block.clear(); + return; + } + + for (const auto i : ext::range(0, block.columns())) + { + auto & col = block.getByPosition(i); + + if (col.column) + col.column = col.column->filter(*filter_and_holder.data, -1); + } +} + MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t max_rows, MarkRanges & ranges) { ReadResult result; @@ -468,29 +556,19 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::startReadingChain(size_t return result; } -void MergeTreeRangeReader::continueReadingChain(ReadResult & result) +Block MergeTreeRangeReader::continueReadingChain(ReadResult & result) { - if (result.numReadRows() == 0) + Block block; + + if (result.rowsPerGranule().empty()) { /// If zero rows were read on prev step, than there is no more rows to read. /// Last granule may have less rows than index_granularity, so finish reading manually. stream.finish(); - return; + return block; } - size_t rows_to_skip_in_last_granule = 0; - - { - size_t rows_in_last_granule = result.rowsPerGranule().back(); - result.optimize(); - rows_to_skip_in_last_granule = rows_in_last_granule - result.rowsPerGranule().back(); - - if (auto & filter = result.getFilter()) - { - if (ConstantFilterDescription(*filter).always_false) - throw Exception("Shouldn't read rows with constant zero prewhere result.", ErrorCodes::LOGICAL_ERROR); - } - } + result.optimize(); auto & rows_per_granule = result.rowsPerGranule(); auto & started_ranges = result.startedRanges(); @@ -504,76 +582,43 @@ void MergeTreeRangeReader::continueReadingChain(ReadResult & result) if (next_range_to_start < started_ranges.size() && i == started_ranges[next_range_to_start].num_granules_read_before_start) { - added_rows += stream.finalize(result.block); + added_rows += stream.finalize(block); auto & range = started_ranges[next_range_to_start].range; ++next_range_to_start; stream = Stream(range.begin, range.end, index_granularity, merge_tree_reader); } bool last = i + 1 == size; - added_rows += stream.read(result.block, rows_per_granule[i], !last); + added_rows += stream.read(block, rows_per_granule[i], !last); } - stream.skip(rows_to_skip_in_last_granule); - added_rows += stream.finalize(result.block); + stream.skip(result.numRowsToSkipInLastGranule()); + added_rows += stream.finalize(block); /// added_rows may be zero if all columns were read in prewhere and it's ok. if (added_rows && added_rows != result.numReadRows()) throw Exception("RangeReader read " + toString(added_rows) + " rows, but " + toString(result.numReadRows()) + " expected.", ErrorCodes::LOGICAL_ERROR); + + return block; } void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - - const auto & columns = merge_tree_reader->getColumns(); - - auto filterColumns = [&result, &columns](const IColumn::Filter & filter) - { - for (const auto & column : columns) - { - if (result.block.has(column.name)) - { - auto & column_with_type_and_name = result.block.getByName(column.name); - if (column_with_type_and_name.column) - column_with_type_and_name.column = column_with_type_and_name.column->filter(filter, -1); - } - } - }; - - auto filterBlock = [&result](const IColumn::Filter & filter) - { - for (const auto i : ext::range(0, result.block.columns())) - { - auto & col = result.block.safeGetByPosition(i); - - if (col.column && col.column->size() == filter.size()) - col.column = col.column->filter(filter, -1); - } - }; - - if (auto & filter = result.getFilter()) - { - ConstantFilterDescription constant_filter_description(*filter); - if (constant_filter_description.always_false) - throw Exception("RangeReader mustn't execute prewhere actions with const zero prewhere result.", - ErrorCodes::LOGICAL_ERROR); - if (!constant_filter_description.always_true) - { - FilterDescription filter_and_holder(*filter); - filterColumns(*filter_and_holder.data); - } - } - - merge_tree_reader->fillMissingColumns(result.block, *ordered_names, always_reorder); - ColumnPtr filter; if (prewhere_actions) { prewhere_actions->execute(result.block); auto & prewhere_column = result.block.getByName(*prewhere_column_name); + size_t rows = result.block.rows(); filter = std::move(prewhere_column.column); - prewhere_column.column = prewhere_column.type->createColumnConst(result.block.rows(), UInt64(1)); + prewhere_column.column = prewhere_column.type->createColumnConst(rows, UInt64(1)); + + ConstantFilterDescription constant_filter_description(*filter); + if (constant_filter_description.always_false) + result.block.clear(); + else if (!constant_filter_description.always_true) + filterBlock(result.block, filter); } if (filter && result.getFilter()) @@ -586,13 +631,6 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (filter) result.setFilter(filter); - - filter = result.getFilter(); - if (filter) - { - FilterDescription filter_and_holder(*filter); - filterBlock(*filter_and_holder.data); - } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 3c10256e90b..959f3eec81f 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -23,7 +23,6 @@ public: size_t numReadRowsInCurrentGranule() const; size_t numPendingRowsInCurrentGranule() const; - size_t numPendingRows() const; bool isCurrentRangeFinished() const; bool isInitialized() const { return is_initialized; } @@ -95,14 +94,18 @@ public: class ReadResult { public: + using NumRows = std::vector; + struct RangeInfo { size_t num_granules_read_before_start; MarkRange range; }; - const std::vector & startedRanges() const { return started_ranges; } - const std::vector & rowsPerGranule() const { return rows_per_granule; } + using RangesInfo = std::vector; + + const RangesInfo & startedRanges() const { return started_ranges; } + const NumRows & rowsPerGranule() const { return rows_per_granule; } /// The number of rows were read at LAST iteration in chain. <= num_added_rows + num_filtered_rows. size_t numReadRows() const { return num_read_rows; } @@ -110,6 +113,7 @@ public: size_t numAddedRows() const { return num_added_rows; } /// The number of filtered rows at all steps in reading chain. size_t numFilteredRows() const { return num_filtered_rows; } + size_t numRowsToSkipInLastGranule() const { return num_rows_to_skip_in_last_granule; } /// The number of bytes read from disk. size_t numBytesRead() const { return num_bytes_read; } /// Filter you need to apply to newly-read columns in order to add them to block. @@ -132,9 +136,9 @@ public: Block block; private: - std::vector started_ranges; + RangesInfo started_ranges; /// The number of rows read from each granule. - std::vector rows_per_granule; + NumRows rows_per_granule; /// Sum(rows_per_granule) size_t num_read_rows = 0; /// The number of rows was added to block while reading columns. May be zero if no read columns present in part. @@ -143,12 +147,15 @@ public: size_t num_filtered_rows = 0; /// Zero if filter is nullptr. size_t num_zeros_in_filter = 0; + /// The number of rows was removed from last granule after clear or optimize. + size_t num_rows_to_skip_in_last_granule = 0; /// Without any filtration. size_t num_bytes_read = 0; /// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= num_read_rows. ColumnPtr filter; - void collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter); + void collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter, const NumRows & zero_tails); + size_t countZeroTails(const IColumn::Filter & filter, NumRows & zero_tails) const; size_t numZerosInFilter() const; static size_t numZerosInTail(const UInt8 * begin, const UInt8 * end); }; @@ -158,8 +165,9 @@ public: private: ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); - void continueReadingChain(ReadResult & result); + Block continueReadingChain(ReadResult & result); void executePrewhereActionsAndFilterColumns(ReadResult & result); + void filterBlock(Block & block, const ColumnPtr & filter) const; size_t index_granularity = 0; MergeTreeReader * merge_tree_reader = nullptr; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 08549cd800b..6efa561e272 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -407,7 +407,7 @@ static bool arrayHasNoElementsRead(const IColumn & column) } -void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_names, bool always_reorder) +void MergeTreeReader::fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults) { if (!res) throw Exception("Empty block passed to fillMissingColumns", ErrorCodes::LOGICAL_ERROR); @@ -435,8 +435,8 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name } } - bool should_evaluate_defaults = false; - bool should_sort = always_reorder; + should_evaluate_missing_defaults = false; + should_reorder = false; size_t rows = res.rows(); @@ -456,10 +456,10 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name if (!has_column) { - should_sort = true; + should_reorder = true; if (storage.column_defaults.count(requested_column.name) != 0) { - should_evaluate_defaults = true; + should_evaluate_missing_defaults = true; continue; } @@ -489,22 +489,40 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name res.insert(std::move(column_to_add)); } } + } + catch (Exception & e) + { + /// Better diagnostics. + e.addMessage("(while reading from part " + path + ")"); + throw; + } +} - /// evaluate defaulted columns if necessary - if (should_evaluate_defaults) - evaluateMissingDefaults(res, columns, storage.column_defaults, storage.context); +void MergeTreeReader::reorderColumns(Block & res, const Names & ordered_names) +{ + try + { + Block ordered_block; - /// sort columns to ensure consistent order among all blocks - if (should_sort) - { - Block ordered_block; + for (const auto & name : ordered_names) + if (res.has(name)) + ordered_block.insert(res.getByName(name)); - for (const auto & name : ordered_names) - if (res.has(name)) - ordered_block.insert(res.getByName(name)); + std::swap(res, ordered_block); + } + catch (Exception & e) + { + /// Better diagnostics. + e.addMessage("(while reading from part " + path + ")"); + throw; + } +} - std::swap(res, ordered_block); - } +void MergeTreeReader::evaluateMissingDefaults(Block & res) +{ + try + { + DB::evaluateMissingDefaults(res, columns, storage.column_defaults, storage.context); } catch (Exception & e) { diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 3ac4365f97c..a4a6b4dc0f7 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -41,7 +41,11 @@ public: /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// If at least one column was added, reorders all columns in the block according to ordered_names. - void fillMissingColumns(Block & res, const Names & ordered_names, const bool always_reorder); + void fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults); + /// Sort columns to ensure consistent order among all blocks. + void reorderColumns(Block & res, const Names & ordered_names); + /// Evaluate defaulted columns if necessary. + void evaluateMissingDefaults(Block & res); const NamesAndTypesList & getColumns() const { return columns; } From b2b1588d5f9f825048b0217aca8cab7c4daf6f1a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 5 Mar 2018 17:25:20 +0300 Subject: [PATCH 16/70] added FilterWithZerosCounter --- .../MergeTree/MergeTreeRangeReader.cpp | 180 +++++++++--------- .../Storages/MergeTree/MergeTreeRangeReader.h | 39 +++- 2 files changed, 122 insertions(+), 97 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index 43313998251..b5c8efb996e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -181,6 +181,53 @@ size_t MergeTreeRangeReader::Stream::finalize(Block & block) } +MergeTreeRangeReader::FilterWithZerosCounter::FilterWithZerosCounter(const ColumnPtr & filter_) +{ + ConstantFilterDescription constant_filter_description(*filter_); + always_false = constant_filter_description.always_false; + always_true = constant_filter_description.always_true; + + if (always_false) + num_zeros = filter_->size(); + + if (!isConstant()) + { + FilterDescription description(*filter_); + holder = description.data_holder ? description.data_holder : filter_; + filter = description.data; + + num_zeros = countBytesInFilter(*filter) - filter->size(); + if (num_zeros == 0) + always_true = true; + else if (num_zeros == filter->size()) + always_false = true; + + if (isConstant()) + { + holder = nullptr; + filter = nullptr; + } + } +} + +const IColumn::Filter & MergeTreeRangeReader::FilterWithZerosCounter::getFilter() const +{ + if (isConstant()) + throw Exception("Cant't return IColumn::Filter from constant column.", ErrorCodes::LOGICAL_ERROR); + + return *filter; +} + +void MergeTreeRangeReader::FilterWithZerosCounter::setFilter(const ColumnPtr & filter_, size_t num_zeros_) +{ + if (isConstant()) + throw Exception("Cant't set filter for constant column.", ErrorCodes::LOGICAL_ERROR); + + holder = filter_; + filter = &static_cast(holder.get())->getData(); +} + + void MergeTreeRangeReader::ReadResult::addGranule(size_t num_rows) { rows_per_granule.push_back(num_rows); @@ -208,49 +255,37 @@ void MergeTreeRangeReader::ReadResult::clear() /// Need to save information about the number of granules. num_rows_to_skip_in_last_granule += rows_per_granule.back(); rows_per_granule.assign(rows_per_granule.size(), 0); - num_filtered_rows += num_read_rows - num_zeros_in_filter; + num_filtered_rows += num_read_rows - filter.numZeros(); num_read_rows = 0; num_added_rows = 0; - num_zeros_in_filter = 0; - filter = nullptr; + filter = FilterWithZerosCounter(); } void MergeTreeRangeReader::ReadResult::optimize() { - if (num_read_rows == 0 || !filter) + if (num_read_rows == 0 || filter.isConstant()) return; - ConstantFilterDescription constant_filter_description(*filter); + NumRows zero_tails; + auto total_zero_rows_in_tails = countZeroTails(filter.getFilter(), zero_tails); - if (constant_filter_description.always_false) - clear(); - else if (constant_filter_description.always_true) - filter = nullptr; - else + /// Just a guess. If only a few rows may be skipped, it's better not to skip at all. + if (2 * total_zero_rows_in_tails > filter.getFilter().size()) { - FilterDescription prev_description(*filter); - NumRows zero_tails; - auto total_zero_rows_in_tails = countZeroTails(*prev_description.data, zero_tails); + auto new_filter = ColumnUInt8::create(filter.getFilter().size() - total_zero_rows_in_tails); + IColumn::Filter & new_data = new_filter->getData(); - /// Just a guess. If only a few rows may be skipped, it's better not to skip at all. - if (2 * total_zero_rows_in_tails > filter->size()) - { + size_t rows_in_last_granule = rows_per_granule.back(); - auto new_filter = ColumnUInt8::create(prev_description.data->size() - total_zero_rows_in_tails); - IColumn::Filter & new_data = new_filter->getData(); + collapseZeroTails(filter.getFilter(), new_data, zero_tails); - size_t rows_in_last_granule = rows_per_granule.back(); + size_t num_removed_zeroes = new_filter->size() - num_read_rows; + num_read_rows = new_filter->size(); + size_t num_zeros_in_filter = filter.numZeros() - num_removed_zeroes; + num_rows_to_skip_in_last_granule += rows_in_last_granule - rows_per_granule.back(); - collapseZeroTails(*prev_description.data, new_data, zero_tails); - - size_t num_removed_zeroes = new_filter->size() - num_read_rows; - num_read_rows = new_filter->size(); - num_zeros_in_filter -= num_removed_zeroes; - num_rows_to_skip_in_last_granule += rows_in_last_granule - rows_per_granule.back(); - - filter = std::move(new_filter); - } + filter.setFilter(new_filter, num_zeros_in_filter); } } @@ -337,60 +372,33 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con return count; } -size_t MergeTreeRangeReader::ReadResult::numZerosInFilter() const +void MergeTreeRangeReader::ReadResult::setFilter(const FilterWithZerosCounter & filter_) { - if (!filter) - return 0; + if (filter_.alwaysTrue() && !filter.alwaysTrue) + throw Exception("Can't replace existing filter with empty.", ErrorCodes::LOGICAL_ERROR); + if (!filter_.alwaysTrue()) { - ConstantFilterDescription constant_filter_description(*filter); - if (constant_filter_description.always_false) - return filter->size(); - if (constant_filter_description.always_true) - return 0; + size_t new_size = filter_.alwaysFalse() ? filter_.numZeros() : filter_.getFilter().size(); + + if (new_size != num_read_rows) + throw Exception("Can't set filter because it's size is " + toString(new_size) + " but " + + toString(num_read_rows) + " rows was read.", ErrorCodes::LOGICAL_ERROR); } - FilterDescription description(*filter); - - auto data = description.data; - auto size = description.data->size(); - - return size - countBytesInFilter(*data); -} - -void MergeTreeRangeReader::ReadResult::setFilter(ColumnPtr filter_) -{ - if (!filter_ && filter) - throw Exception("Can't remove exising filter with empty.", ErrorCodes::LOGICAL_ERROR); - - if (!filter_) - return; - - if (filter_->size() < num_read_rows) - throw Exception("Can't set filter because it's size is " + toString(filter_->size()) + " but " - + toString(num_read_rows) + " rows was read.", ErrorCodes::LOGICAL_ERROR); - - if (filter && filter_->size() != filter->size()) - throw Exception("Can't set filter because it's size is " + toString(filter_->size()) + " but previous filter" - + " has size " + toString(filter->size()) + ".", ErrorCodes::LOGICAL_ERROR); - - filter = std::move(filter_); - - size_t num_zeros = numZerosInFilter(); - - if (num_zeros < num_zeros_in_filter) + if (filter_.numZeros() < filter.numZeros()) throw Exception("New filter has less zeros than previous.", ErrorCodes::LOGICAL_ERROR); - if (num_zeros == filter->size()) + size_t num_zeros_in_prev_filter = filter.numZeros(); + filter = std::move(filter_); + + if (filter.alwaysFalse()) clear(); - else if (num_zeros == 0) - filter = nullptr; else { - size_t added_zeros = num_zeros - num_zeros_in_filter; + size_t added_zeros = filter.numZeros() - num_zeros_in_prev_filter; num_added_rows -= added_zeros; num_filtered_rows += added_zeros; - num_zeros_in_filter = num_zeros; } } @@ -453,9 +461,7 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar /// Fill missing columns before filtering because some arrays from Nested may have empty data. merge_tree_reader->fillMissingColumns(block, should_reorder, should_evaluate_missing_defaults); - const auto & filter = read_result.getFilter(); - if (filter) - filterBlock(block, filter); + filterBlock(block, read_result.getFilter()); for (auto i : ext::range(0, block.columns())) read_result.block.insert(std::move(block.getByPosition(i))); @@ -496,28 +502,25 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar return read_result; } -void MergeTreeRangeReader::filterBlock(Block & block, const ColumnPtr & filter) const +void MergeTreeRangeReader::filterBlock(Block & block, const FilterWithZerosCounter & filter) const { - if (!filter) + if (filter.alwaysTrue()) return; - FilterDescription filter_and_holder(*filter); - - auto bytes_in_filter = countBytesInFilter(*filter_and_holder.data); - if (bytes_in_filter == filter->size()) - return; - else if (bytes_in_filter == 0) + if (filter.alwaysFalse()) { block.clear(); return; } + const auto & filter_data = filter.getFilter(); + for (const auto i : ext::range(0, block.columns())) { auto & col = block.getByPosition(i); if (col.column) - col.column = col.column->filter(*filter_and_holder.data, -1); + col.column = col.column->filter(filter_data, -1); } } @@ -605,23 +608,22 @@ Block MergeTreeRangeReader::continueReadingChain(ReadResult & result) void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - ColumnPtr filter; + FilterWithZerosCounter filter; if (prewhere_actions) { prewhere_actions->execute(result.block); auto & prewhere_column = result.block.getByName(*prewhere_column_name); size_t rows = result.block.rows(); - filter = std::move(prewhere_column.column); + filter = FilterWithZerosCounter(prewhere_column.column); prewhere_column.column = prewhere_column.type->createColumnConst(rows, UInt64(1)); - ConstantFilterDescription constant_filter_description(*filter); - if (constant_filter_description.always_false) + if (filter.alwaysFalse()) result.block.clear(); - else if (!constant_filter_description.always_true) + else if (!filter.alwaysTrue()) filterBlock(result.block, filter); } - if (filter && result.getFilter()) + if (!filter.alwaysTrue() && !result.getFilter().alwaysTrue()) { /// TODO: implement for prewhere chain. /// In order to do it we need combine filter and result.filter, where filter filters only '1' in result.filter. @@ -629,7 +631,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r ErrorCodes::LOGICAL_ERROR); } - if (filter) + if (!filter.alwaysTrue()) result.setFilter(filter); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 959f3eec81f..818a2acfc4e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -90,6 +90,32 @@ public: size_t readRows(Block & block, size_t num_rows); }; + class FilterWithZerosCounter + { + public: + /// By default, filter is null and has always_true status. + FilterWithZerosCounter() = default; + explicit FilterWithZerosCounter(const ColumnPtr & filter); + + /// Can be used only if isConstant(). + const IColumn::Filter & getFilter() const; + size_t numZeros() const { return num_zeros; } + + bool alwaysTrue() const { return always_true; } + bool alwaysFalse() const { return always_false; } + bool isConstant() const { return always_false || always_true; } + + void setFilter(const ColumnPtr & filter, size_t num_zeros_); + + private: + ColumnPtr holder; + const IColumn::Filter * filter; + size_t num_zeros = 0; + + bool always_true = true; + bool always_false = false; + }; + /// Statistics after next reading step. class ReadResult { @@ -117,7 +143,7 @@ public: /// The number of bytes read from disk. size_t numBytesRead() const { return num_bytes_read; } /// Filter you need to apply to newly-read columns in order to add them to block. - const ColumnPtr & getFilter() const { return filter; } + const FilterWithZerosCounter & getFilter() const { return filter; } void addGranule(size_t num_rows); void adjustLastGranule(); @@ -125,7 +151,7 @@ public: void addRange(const MarkRange & range) { started_ranges.push_back({rows_per_granule.size(), range}); } /// Set filter or replace old one. Filter must have more zeroes than previous. - void setFilter(ColumnPtr filter_); + void setFilter(const FilterWithZerosCounter & filter_); /// For each granule calculate the number of filtered rows at the end. Remove them and update filter. void optimize(); /// Remove all rows from granules. @@ -145,18 +171,15 @@ public: size_t num_added_rows = 0; /// num_zeros_in_filter + the number of rows removed after optimizes. size_t num_filtered_rows = 0; - /// Zero if filter is nullptr. - size_t num_zeros_in_filter = 0; /// The number of rows was removed from last granule after clear or optimize. size_t num_rows_to_skip_in_last_granule = 0; /// Without any filtration. size_t num_bytes_read = 0; - /// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= num_read_rows. - ColumnPtr filter; + /// alwaysTrue() if prev reader hasn't prewhere_actions. Otherwise filter.size() >= num_read_rows. + FilterWithZerosCounter filter; void collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter, const NumRows & zero_tails); size_t countZeroTails(const IColumn::Filter & filter, NumRows & zero_tails) const; - size_t numZerosInFilter() const; static size_t numZerosInTail(const UInt8 * begin, const UInt8 * end); }; @@ -167,7 +190,7 @@ private: ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); Block continueReadingChain(ReadResult & result); void executePrewhereActionsAndFilterColumns(ReadResult & result); - void filterBlock(Block & block, const ColumnPtr & filter) const; + void filterBlock(Block & block, const FilterWithZerosCounter & filter) const; size_t index_granularity = 0; MergeTreeReader * merge_tree_reader = nullptr; From 532b57aef8832b0cb85eeb5613f6a9b03a8da9cd Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 5 Mar 2018 17:41:43 +0300 Subject: [PATCH 17/70] perf. fixes --- .../MergeTreeBaseBlockInputStream.cpp | 45 ++-- .../MergeTree/MergeTreeRangeReader.cpp | 233 +++++++++--------- .../Storages/MergeTree/MergeTreeRangeReader.h | 64 ++--- 3 files changed, 160 insertions(+), 182 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 21ca7a2c9f8..0bc13cb0c10 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -119,19 +119,30 @@ Block MergeTreeBaseBlockInputStream::readFromPart() { if (prewhere_actions) { - task->pre_range_reader = MergeTreeRangeReader( - pre_reader.get(), index_granularity, nullptr, prewhere_actions, - &prewhere_column_name, &task->ordered_names, task->should_reorder); + if (reader->getColumns().empty()) + { + task->range_reader = MergeTreeRangeReader( + pre_reader.get(), index_granularity, nullptr, prewhere_actions, + &prewhere_column_name, &task->ordered_names, + task->should_reorder, task->remove_prewhere_column, true); + } + else + { + task->pre_range_reader = MergeTreeRangeReader( + pre_reader.get(), index_granularity, nullptr, prewhere_actions, + &prewhere_column_name, &task->ordered_names, + task->should_reorder, task->remove_prewhere_column, false); - task->range_reader = MergeTreeRangeReader( - reader.get(), index_granularity, &task->pre_range_reader, nullptr, - nullptr, &task->ordered_names, true); + task->range_reader = MergeTreeRangeReader( + reader.get(), index_granularity, &task->pre_range_reader, nullptr, + nullptr, &task->ordered_names, true, false, true); + } } else { task->range_reader = MergeTreeRangeReader( - reader.get(), index_granularity, nullptr, nullptr, - nullptr, &task->ordered_names, task->should_reorder); + reader.get(), index_granularity, nullptr, prewhere_actions, + nullptr, &task->ordered_names, task->should_reorder, false, true); } } @@ -145,27 +156,23 @@ Block MergeTreeBaseBlockInputStream::readFromPart() if (read_result.block.rows() == 0) read_result.block.clear(); - size_t rows_read = read_result.numAddedRows() + read_result.numFilteredRows(); + size_t num_filtered_rows = read_result.numReadRows() - read_result.block.rows(); - progressImpl({ rows_read, read_result.numBytesRead() }); + progressImpl({ read_result.numReadRows(), read_result.numBytesRead() }); if (task->size_predictor) { - task->size_predictor->updateFilteredRowsRation(rows_read, read_result.numFilteredRows()); + task->size_predictor->updateFilteredRowsRation(read_result.numReadRows(), num_filtered_rows); if (read_result.block) task->size_predictor->update(read_result.block); } - if (!prewhere_column_name.empty() && read_result.block.has(prewhere_column_name)) + if (prewhere_actions && !task->remove_prewhere_column) { - if (task->remove_prewhere_column) - read_result.block.erase(prewhere_column_name); - else - { - auto & column = read_result.block.getByName(prewhere_column_name); - column.column = column.column->convertToFullColumnIfConst(); - } + /// Convert const column to full here because it's cheaper to filter const column than full. + auto & column = read_result.block.getByName(prewhere_column_name); + column.column = column.column->convertToFullColumnIfConst(); } read_result.block.checkNumberOfRows(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index b5c8efb996e..0a1e2395696 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -181,62 +181,15 @@ size_t MergeTreeRangeReader::Stream::finalize(Block & block) } -MergeTreeRangeReader::FilterWithZerosCounter::FilterWithZerosCounter(const ColumnPtr & filter_) -{ - ConstantFilterDescription constant_filter_description(*filter_); - always_false = constant_filter_description.always_false; - always_true = constant_filter_description.always_true; - - if (always_false) - num_zeros = filter_->size(); - - if (!isConstant()) - { - FilterDescription description(*filter_); - holder = description.data_holder ? description.data_holder : filter_; - filter = description.data; - - num_zeros = countBytesInFilter(*filter) - filter->size(); - if (num_zeros == 0) - always_true = true; - else if (num_zeros == filter->size()) - always_false = true; - - if (isConstant()) - { - holder = nullptr; - filter = nullptr; - } - } -} - -const IColumn::Filter & MergeTreeRangeReader::FilterWithZerosCounter::getFilter() const -{ - if (isConstant()) - throw Exception("Cant't return IColumn::Filter from constant column.", ErrorCodes::LOGICAL_ERROR); - - return *filter; -} - -void MergeTreeRangeReader::FilterWithZerosCounter::setFilter(const ColumnPtr & filter_, size_t num_zeros_) -{ - if (isConstant()) - throw Exception("Cant't set filter for constant column.", ErrorCodes::LOGICAL_ERROR); - - holder = filter_; - filter = &static_cast(holder.get())->getData(); -} - - void MergeTreeRangeReader::ReadResult::addGranule(size_t num_rows) { rows_per_granule.push_back(num_rows); - num_read_rows += num_rows; + total_rows_per_granule += num_rows; } void MergeTreeRangeReader::ReadResult::adjustLastGranule() { - size_t num_rows_to_subtract = num_read_rows - num_added_rows; + size_t num_rows_to_subtract = total_rows_per_granule - num_read_rows; if (rows_per_granule.empty()) throw Exception("Can't adjust last granule because no granules were added.", ErrorCodes::LOGICAL_ERROR); @@ -247,7 +200,7 @@ void MergeTreeRangeReader::ReadResult::adjustLastGranule() ErrorCodes::LOGICAL_ERROR); rows_per_granule.back() -= num_rows_to_subtract; - num_read_rows -= num_rows_to_subtract; + total_rows_per_granule -= num_rows_to_subtract; } void MergeTreeRangeReader::ReadResult::clear() @@ -255,37 +208,47 @@ void MergeTreeRangeReader::ReadResult::clear() /// Need to save information about the number of granules. num_rows_to_skip_in_last_granule += rows_per_granule.back(); rows_per_granule.assign(rows_per_granule.size(), 0); - num_filtered_rows += num_read_rows - filter.numZeros(); - num_read_rows = 0; - num_added_rows = 0; - filter = FilterWithZerosCounter(); + total_rows_per_granule = 0; + filter_holder = nullptr; + filter = nullptr; } void MergeTreeRangeReader::ReadResult::optimize() { - if (num_read_rows == 0 || filter.isConstant()) + if (total_rows_per_granule == 0 || filter == nullptr) return; NumRows zero_tails; - auto total_zero_rows_in_tails = countZeroTails(filter.getFilter(), zero_tails); + auto total_zero_rows_in_tails = countZeroTails(filter->getData(), zero_tails); + + if (total_zero_rows_in_tails == filter->size()) + { + clear(); + return; + } + else if (total_zero_rows_in_tails == 0 && countBytesInFilter(filter->getData()) == filter->size()) + { + filter_holder = nullptr; + filter = nullptr; + return; + } /// Just a guess. If only a few rows may be skipped, it's better not to skip at all. - if (2 * total_zero_rows_in_tails > filter.getFilter().size()) + if (2 * total_zero_rows_in_tails > filter->size()) { - auto new_filter = ColumnUInt8::create(filter.getFilter().size() - total_zero_rows_in_tails); + auto new_filter = ColumnUInt8::create(filter->size() - total_zero_rows_in_tails); IColumn::Filter & new_data = new_filter->getData(); size_t rows_in_last_granule = rows_per_granule.back(); - collapseZeroTails(filter.getFilter(), new_data, zero_tails); + collapseZeroTails(filter->getData(), new_data, zero_tails); - size_t num_removed_zeroes = new_filter->size() - num_read_rows; - num_read_rows = new_filter->size(); - size_t num_zeros_in_filter = filter.numZeros() - num_removed_zeroes; + total_rows_per_granule = new_filter->size(); num_rows_to_skip_in_last_granule += rows_in_last_granule - rows_per_granule.back(); - filter.setFilter(new_filter, num_zeros_in_filter); + filter = new_filter.get(); + filter_holder = std::move(new_filter); } } @@ -372,33 +335,30 @@ size_t MergeTreeRangeReader::ReadResult::numZerosInTail(const UInt8 * begin, con return count; } -void MergeTreeRangeReader::ReadResult::setFilter(const FilterWithZerosCounter & filter_) +void MergeTreeRangeReader::ReadResult::setFilter(const ColumnPtr & new_filter) { - if (filter_.alwaysTrue() && !filter.alwaysTrue) + if (!new_filter && filter) throw Exception("Can't replace existing filter with empty.", ErrorCodes::LOGICAL_ERROR); - if (!filter_.alwaysTrue()) + if (filter) { - size_t new_size = filter_.alwaysFalse() ? filter_.numZeros() : filter_.getFilter().size(); + size_t new_size = new_filter->size(); - if (new_size != num_read_rows) + if (new_size != total_rows_per_granule) throw Exception("Can't set filter because it's size is " + toString(new_size) + " but " - + toString(num_read_rows) + " rows was read.", ErrorCodes::LOGICAL_ERROR); + + toString(total_rows_per_granule) + " rows was read.", ErrorCodes::LOGICAL_ERROR); } - if (filter_.numZeros() < filter.numZeros()) - throw Exception("New filter has less zeros than previous.", ErrorCodes::LOGICAL_ERROR); - - size_t num_zeros_in_prev_filter = filter.numZeros(); - filter = std::move(filter_); - - if (filter.alwaysFalse()) + ConstantFilterDescription const_description(*new_filter); + if (const_description.always_false) clear(); - else + else if (!const_description.always_true) { - size_t added_zeros = filter.numZeros() - num_zeros_in_prev_filter; - num_added_rows -= added_zeros; - num_filtered_rows += added_zeros; + FilterDescription filter_description(*new_filter); + filter_holder = filter_description.data_holder ? filter_description.data_holder : new_filter; + filter = typeid_cast(filter_holder.get()); + if (!filter) + throw Exception("setFilter function expected ColumnUInt8.", ErrorCodes::LOGICAL_ERROR); } } @@ -406,11 +366,13 @@ void MergeTreeRangeReader::ReadResult::setFilter(const FilterWithZerosCounter & MergeTreeRangeReader::MergeTreeRangeReader( MergeTreeReader * merge_tree_reader, size_t index_granularity, MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions, - const String * prewhere_column_name, const Names * ordered_names, bool always_reorder) + const String * prewhere_column_name, const Names * ordered_names, + bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain) : index_granularity(index_granularity), merge_tree_reader(merge_tree_reader) - , prev_reader(prev_reader), prewhere_actions(std::move(prewhere_actions)) - , prewhere_column_name(prewhere_column_name), ordered_names(ordered_names) - , always_reorder(always_reorder), is_initialized(true) + , prev_reader(prev_reader), prewhere_column_name(prewhere_column_name) + , ordered_names(ordered_names), prewhere_actions(std::move(prewhere_actions)) + , always_reorder(always_reorder), remove_prewhere_column(remove_prewhere_column) + , last_reader_in_chain(last_reader_in_chain), is_initialized(true) { } @@ -428,7 +390,7 @@ size_t MergeTreeRangeReader::numPendingRowsInCurrentGranule() const if (prev_reader) return prev_reader->numPendingRowsInCurrentGranule(); - auto pending_rows = stream.numPendingRowsInCurrentGranule(); + auto pending_rows = stream.numPendingRowsInCurrentGranule(); /// If pending_rows is zero, than stream is not initialized. return pending_rows ? pending_rows : index_granularity; } @@ -461,7 +423,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar /// Fill missing columns before filtering because some arrays from Nested may have empty data. merge_tree_reader->fillMissingColumns(block, should_reorder, should_evaluate_missing_defaults); - filterBlock(block, read_result.getFilter()); + if (read_result.getFilter()) + filterBlock(block, read_result.getFilter()->getData()); for (auto i : ext::range(0, block.columns())) read_result.block.insert(std::move(block.getByPosition(i))); @@ -502,25 +465,22 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar return read_result; } -void MergeTreeRangeReader::filterBlock(Block & block, const FilterWithZerosCounter & filter) const +void MergeTreeRangeReader::filterBlock(Block & block, const IColumn::Filter & filter) const { - if (filter.alwaysTrue()) - return; - - if (filter.alwaysFalse()) - { - block.clear(); - return; - } - - const auto & filter_data = filter.getFilter(); - for (const auto i : ext::range(0, block.columns())) { auto & col = block.getByPosition(i); if (col.column) - col.column = col.column->filter(filter_data, -1); + { + col.column = col.column->filter(filter, -1); + + if (col.column->empty()) + { + block.clear(); + return; + } + } } } @@ -571,8 +531,6 @@ Block MergeTreeRangeReader::continueReadingChain(ReadResult & result) return block; } - result.optimize(); - auto & rows_per_granule = result.rowsPerGranule(); auto & started_ranges = result.startedRanges(); @@ -599,31 +557,25 @@ Block MergeTreeRangeReader::continueReadingChain(ReadResult & result) added_rows += stream.finalize(block); /// added_rows may be zero if all columns were read in prewhere and it's ok. - if (added_rows && added_rows != result.numReadRows()) + if (added_rows && added_rows != result.totalRowsPerGranule()) throw Exception("RangeReader read " + toString(added_rows) + " rows, but " - + toString(result.numReadRows()) + " expected.", ErrorCodes::LOGICAL_ERROR); + + toString(result.totalRowsPerGranule()) + " expected.", ErrorCodes::LOGICAL_ERROR); return block; } void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & result) { - FilterWithZerosCounter filter; - if (prewhere_actions) - { - prewhere_actions->execute(result.block); - auto & prewhere_column = result.block.getByName(*prewhere_column_name); - size_t rows = result.block.rows(); - filter = FilterWithZerosCounter(prewhere_column.column); - prewhere_column.column = prewhere_column.type->createColumnConst(rows, UInt64(1)); + if (!prewhere_actions) + return; - if (filter.alwaysFalse()) - result.block.clear(); - else if (!filter.alwaysTrue()) - filterBlock(result.block, filter); - } + prewhere_actions->execute(result.block); + auto & prewhere_column = result.block.getByName(*prewhere_column_name); + size_t prev_rows = result.block.rows(); + ColumnPtr filter = prewhere_column.column; + prewhere_column.column = nullptr; - if (!filter.alwaysTrue() && !result.getFilter().alwaysTrue()) + if (result.getFilter()) { /// TODO: implement for prewhere chain. /// In order to do it we need combine filter and result.filter, where filter filters only '1' in result.filter. @@ -631,8 +583,51 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r ErrorCodes::LOGICAL_ERROR); } - if (!filter.alwaysTrue()) - result.setFilter(filter); + result.setFilter(filter); + if (!last_reader_in_chain) + result.optimize(); + + bool filter_always_true = !result.getFilter() && result.totalRowsPerGranule() == filter->size(); + + if (result.totalRowsPerGranule() == 0) + result.block.clear(); + else if (!filter_always_true) + { + FilterDescription filter_description(*filter); + + if (last_reader_in_chain) + { + size_t num_bytes_in_filter = countBytesInFilter(*filter_description.data); + if (num_bytes_in_filter == 0) + result.block.clear(); + else if (num_bytes_in_filter == filter->size()) + filter_always_true = true; + } + + if (!filter_always_true) + filterBlock(result.block, *filter_description.data); + } + + if (!result.block) + return; + + if (remove_prewhere_column) + result.block.erase(*prewhere_column_name); + else + { + /// Calculate the number of rows in block in order to create const column. + size_t rows = result.block.rows(); + /// If block has single column, it's filter. We need to count bytes in it in order to get the number of rows. + if (result.block.columns() == 1) + { + if (result.getFilter()) + rows = countBytesInFilter(result.getFilter()->getData()); + else + rows = prev_rows; + } + + prewhere_column.column = prewhere_column.type->createColumnConst(rows, UInt64(1)); + } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 818a2acfc4e..afba6715c0a 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -5,6 +5,10 @@ namespace DB { +template +class ColumnVector; +using ColumnUInt8 = ColumnVector; + class MergeTreeReader; /// MergeTreeReader iterator which allows sequential reading for arbitrary number of rows between pairs of marks in the same part. @@ -15,7 +19,8 @@ class MergeTreeRangeReader public: MergeTreeRangeReader(MergeTreeReader * merge_tree_reader, size_t index_granularity, MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions, - const String * prewhere_column_name, const Names * ordered_names, bool always_reorder); + const String * prewhere_column_name, const Names * ordered_names, + bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain); MergeTreeRangeReader() = default; @@ -57,7 +62,6 @@ public: class Stream { - public: Stream() = default; Stream(size_t from_mark, size_t to_mark, size_t index_granularity, MergeTreeReader * merge_tree_reader); @@ -90,32 +94,6 @@ public: size_t readRows(Block & block, size_t num_rows); }; - class FilterWithZerosCounter - { - public: - /// By default, filter is null and has always_true status. - FilterWithZerosCounter() = default; - explicit FilterWithZerosCounter(const ColumnPtr & filter); - - /// Can be used only if isConstant(). - const IColumn::Filter & getFilter() const; - size_t numZeros() const { return num_zeros; } - - bool alwaysTrue() const { return always_true; } - bool alwaysFalse() const { return always_false; } - bool isConstant() const { return always_false || always_true; } - - void setFilter(const ColumnPtr & filter, size_t num_zeros_); - - private: - ColumnPtr holder; - const IColumn::Filter * filter; - size_t num_zeros = 0; - - bool always_true = true; - bool always_false = false; - }; - /// Statistics after next reading step. class ReadResult { @@ -134,24 +112,22 @@ public: const NumRows & rowsPerGranule() const { return rows_per_granule; } /// The number of rows were read at LAST iteration in chain. <= num_added_rows + num_filtered_rows. - size_t numReadRows() const { return num_read_rows; } + size_t totalRowsPerGranule() const { return total_rows_per_granule; } /// The number of rows were added to block as a result of reading chain. - size_t numAddedRows() const { return num_added_rows; } - /// The number of filtered rows at all steps in reading chain. - size_t numFilteredRows() const { return num_filtered_rows; } + size_t numReadRows() const { return num_read_rows; } size_t numRowsToSkipInLastGranule() const { return num_rows_to_skip_in_last_granule; } /// The number of bytes read from disk. size_t numBytesRead() const { return num_bytes_read; } /// Filter you need to apply to newly-read columns in order to add them to block. - const FilterWithZerosCounter & getFilter() const { return filter; } + const ColumnUInt8 * getFilter() const { return filter; } void addGranule(size_t num_rows); void adjustLastGranule(); - void addRows(size_t rows) { num_added_rows += rows; } + void addRows(size_t rows) { num_read_rows += rows; } void addRange(const MarkRange & range) { started_ranges.push_back({rows_per_granule.size(), range}); } /// Set filter or replace old one. Filter must have more zeroes than previous. - void setFilter(const FilterWithZerosCounter & filter_); + void setFilter(const ColumnPtr & new_filter); /// For each granule calculate the number of filtered rows at the end. Remove them and update filter. void optimize(); /// Remove all rows from granules. @@ -166,17 +142,16 @@ public: /// The number of rows read from each granule. NumRows rows_per_granule; /// Sum(rows_per_granule) + size_t total_rows_per_granule = 0; + /// The number of rows was read at first step. May be zero if no read columns present in part. size_t num_read_rows = 0; - /// The number of rows was added to block while reading columns. May be zero if no read columns present in part. - size_t num_added_rows = 0; - /// num_zeros_in_filter + the number of rows removed after optimizes. - size_t num_filtered_rows = 0; /// The number of rows was removed from last granule after clear or optimize. size_t num_rows_to_skip_in_last_granule = 0; /// Without any filtration. size_t num_bytes_read = 0; - /// alwaysTrue() if prev reader hasn't prewhere_actions. Otherwise filter.size() >= num_read_rows. - FilterWithZerosCounter filter; + /// nullptr if prev reader hasn't prewhere_actions. Otherwise filter.size() >= total_rows_per_granule. + ColumnPtr filter_holder; + const ColumnUInt8 * filter = nullptr; void collapseZeroTails(const IColumn::Filter & filter, IColumn::Filter & new_filter, const NumRows & zero_tails); size_t countZeroTails(const IColumn::Filter & filter, NumRows & zero_tails) const; @@ -190,20 +165,21 @@ private: ReadResult startReadingChain(size_t max_rows, MarkRanges & ranges); Block continueReadingChain(ReadResult & result); void executePrewhereActionsAndFilterColumns(ReadResult & result); - void filterBlock(Block & block, const FilterWithZerosCounter & filter) const; + void filterBlock(Block & block, const IColumn::Filter & filter) const; size_t index_granularity = 0; MergeTreeReader * merge_tree_reader = nullptr; - MergeTreeRangeReader * prev_reader = nullptr; /// If not nullptr, read from prev_reader firstly. - ExpressionActionsPtr prewhere_actions = nullptr; /// If not nullptr, calculate filter. const String * prewhere_column_name = nullptr; const Names * ordered_names = nullptr; + ExpressionActionsPtr prewhere_actions = nullptr; /// If not nullptr, calculate filter. Stream stream; bool always_reorder = true; + bool remove_prewhere_column = false; + bool last_reader_in_chain = false; bool is_initialized = false; }; From 30196169a128a44ade90e619b5bd131c0860b245 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Mar 2018 18:23:06 +0300 Subject: [PATCH 18/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 390 ++++++++++++++++++++ dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 251 +++++++++++++ 2 files changed, 641 insertions(+) create mode 100644 dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp create mode 100644 dbms/src/Common/ZooKeeper/ZooKeeperImpl.h diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp new file mode 100644 index 00000000000..7f5bb113623 --- /dev/null +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -0,0 +1,390 @@ +#include +#include + +#include +#include +#include + +#include +#include + +#include + + +/** ZooKeeper wire protocol. + +Debugging example: +strace -f -e trace=network -s1000 -x ./clickhouse-zookeeper-cli localhost:2181 + +All numbers are in network byte order (big endian). Sizes are 32 bit. Numbers are signed. + +zxid - incremental transaction number at server side. +xid - unique request number at client side. + +Client connects to one of the specified hosts. +Client sends: + +int32_t sizeof_connect_req; \x00\x00\x00\x2c (44 bytes) + +struct connect_req +{ + int32_t protocolVersion; \x00\x00\x00\x00 (Currently zero) + int64_t lastZxidSeen; \x00\x00\x00\x00\x00\x00\x00\x00 (Zero at first connect) + int32_t timeOut; \x00\x00\x75\x30 (Session timeout in milliseconds: 30000) + int64_t sessionId; \x00\x00\x00\x00\x00\x00\x00\x00 (Zero at first connect) + int32_t passwd_len; \x00\x00\x00\x10 (16) + char passwd[16]; \x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00 (Zero at first connect) +}; + +Server replies: + +struct prime_struct +{ + int32_t len; \x00\x00\x00\x24 (36 bytes) + int32_t protocolVersion; \x00\x00\x00\x00 + int32_t timeOut; \x00\x00\x75\x30 + int64_t sessionId; \x01\x62\x2c\x3d\x82\x43\x00\x27 + int32_t passwd_len; \x00\x00\x00\x10 + char passwd[16]; \x3b\x8c\xe0\xd4\x1f\x34\xbc\x88\x9c\xa7\x68\x69\x78\x64\x98\xe9 +}; + +Client remembers session id and session password. + + +Client may send authentication request (optional). + + +Each one third of timeout, client sends heartbeat: + +int32_t length_of_heartbeat_request \x00\x00\x00\x08 (8) +int32_t ping_xid \xff\xff\xff\xfe (-2, constant) +int32_t ping_op \x00\x00\x00\x0b ZOO_PING_OP 11 + +Server replies: + +int32_t length_of_heartbeat_response \x00\x00\x00\x10 +int32_t ping_xid \xff\xff\xff\xfe +int64 zxid \x00\x00\x00\x00\x00\x01\x87\x98 (incremental server generated number) +int32_t err \x00\x00\x00\x00 + + +Client sends requests. For example, create persistent node '/hello' with value 'world'. + +int32_t request_length \x00\x00\x00\x3a +int32_t xid \x5a\xad\x72\x3f Arbitary number. Used for identification of requests/responses. + libzookeeper uses unix timestamp for first xid and then autoincrement to that value. +int32_t type \x00\x00\x00\x01 ZOO_CREATE_OP 1 +int32_t path_length \x00\x00\x00\x06 +path \x2f\x68\x65\x6c\x6c\x6f /hello +int32_t data_length \x00\x00\x00\x05 +data \x77\x6f\x72\x6c\x64 world +ACLs: + int32_t num_acls \x00\x00\x00\x01 + ACL: + int32_t permissions \x00\x00\x00\x1f + string scheme \x00\x00\x00\x05 + \x77\x6f\x72\x6c\x64 world + string id \x00\x00\x00\x06 + \x61\x6e\x79\x6f\x6e\x65 anyone +int32_t flags \x00\x00\x00\x00 + +Server replies: + +int32_t response_length \x00\x00\x00\x1a +int32_t xid \x5a\xad\x72\x3f +int64 zxid \x00\x00\x00\x00\x00\x01\x87\x99 +int32_t err \x00\x00\x00\x00 +string path_created \x00\x00\x00\x06 + \x2f\x68\x65\x6c\x6c\x6f /hello - may differ to original path in case of sequential nodes. + */ + + +namespace ZooKeeperImpl +{ + +using namespace DB; + +/// Assuming we are at little endian. + +void write(int64_t x, WriteBuffer & out) +{ + x = __builtin_bswap64(x); + writeBinary(x, out); +} + +void write(int32_t x, WriteBuffer & out) +{ + x = __builtin_bswap32(x); + writeBinary(x, out); +} + +void write(const String & s, WriteBuffer & out) +{ + write(int32_t(s.size())); + out.write(s.data(), s.size()); +} + +template void write(char s[N], WriteBuffer & out) +{ + write(int32_t(N)); + out.write(s, N); +} + +template void write(const std::vector & arr, WriteBuffer & out) +{ + write(int32_t(arr.size())); + for (const auto & elem : arr) + write(elem); +} + +void write(const ZooKeeper::ACL & acl, WriteBuffer & out) +{ + acl.write(out); +} + + +void read(int64_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap64(x); +} + +void read(int32_t & x, ReadBuffer & in) +{ + readBinary(x, in); + x = __builtin_bswap32(x); +} + +void read(String & s, ReadBuffer & in) +{ + static constexpr int32_t max_string_size = 1 << 20; + int32_t size = 0; + read(size); + if (size > max_string_size) + throw Exception("Too large string size"); /// TODO error code + s.resize(size); + in.read(&s[0], size); +} + +template void read(char(&arr)[N], ReadBuffer & in) +{ + int32_t size = 0; + read(size); + if (size != N) + throw Exception("Unexpected array size"); /// TODO error code + in.read(arr, N); +} + +void read(ZooKeeper::Stat & stat, ReadBuffer & in) +{ + stat.read(in); +} + + +template +void ZooKeeper::write(const T & x) +{ + write(x, out); +} + +template +void ZooKeeper::read(T & x) +{ + read(x, in); +} + + + +ZooKeeper::~ZooKeeper() +{ + stop = true; + + if (send_thread.joinable()) + send_thread.join(); + + if (receive_thread.joinable()) + receive_thread.join(); +} + + +ZooKeeper::ZooKeeper( + const Addresses & addresses, + const String & root_path, + const String & auth_scheme, + const String & auth_data, + Poco::Timespan session_timeout, + Poco::Timespan connection_timeout) + : root_path(root_path), + session_timeout(session_timeout) +{ + connect(); + + sendHandshake(); + receiveHandshake(); + + if (!auth_scheme.empty()) + sendAuth(-1, auth_scheme, auth_data); + + send_thread = std::thread([this] { sendThread(); }); + receive_thread = std::thread([this] { receiveThread(); }); +} + + +void ZooKeeper::connect( + const Addresses & addresses) +{ + static constexpr size_t num_tries = 3; + bool connected = false; + + for (size_t try_no = 0; try_no < num_tries; ++try_no) + { + for (const auto & address : addresses) + { + try + { + socket.connect(address, connection_timeout); + connected = true; + break; + } + catch (const Poco::Net::NetException & e) + { + /// TODO log exception + } + catch (const Poco::TimeoutException & e) + { + } + } + + if (connected) + break; + } + + if (!connected) + throw Exception("All connection tries failed"); /// TODO more info; error code + + socket->setReceiveTimeout(session_timeout); + socket->setSendTimeout(session_timeout); + socket->setNoDelay(true); + + in.emplace(socket); + out.emplace(socket); +} + + +void ZooKeeper::sendHandshake() +{ + int32_t handshake_length = 44; + int32_t protocol_version = 0; + int64_t last_zxid_seen = 0; + int32_t timeout = session_timeout.totalMilliseconds(); + int64_t session_id = 0; + constexpr int32_t passwd_len = 16; + char passwd[passwd_len] {}; + + write(handshake_length); + write(protocol_version); + write(last_zxid_seen); + write(timeout); + write(session_id); + write(passwd); + + out.next(); +} + + +void ZooKeeper::receiveHandshake() +{ + int32_t handshake_length; + int32_t protocol_version; + int32_t timeout; + int64_t session_id; + constexpr int32_t passwd_len = 16; + char passwd[passwd_len] {}; + + read(handshake_length); + if (handshake_length != 36) + throw Exception("Unexpected handshake length received: " + toString(handshake_length)); + + read(protocol_version); + if (protocol_version != 0) + throw Exception("Unexpected protocol version: " + toString(protocol_version)); + + read(timeout); + if (timeout != session_timeout.totalMilliseconds()) + throw Exception("Received different session timeout from server: " + toString(timeout)); + + read(session_id); + read(passwd); +} + + +void ZooKeeper::sendAuth(XID xid, const String & auth_scheme, const String & auth_data) +{ + // TODO +} + + +void ZooKeeper::sendThread() +{ + XID xid = 0; /// TODO deal with xid overflow + auto prev_heartbeat_time = std::chrono::steady_clock::now(); + + try + { + while (!stop) + { + ++xid; + + auto now = std::chrono::steady_clock::now(); + auto next_heartbeat_time = prev_heartbeat_time + std::chrono::duration(session_timeout.totalMilliseconds()); + auto max_wait = next_heartbeat_time > now + ? std::chrono::duration_cast(next_heartbeat_time - now) + : 0; + + RequestPtr request; + if (requests.tryPop(request, max_wait)) + { + sendRequest(xid, *request); + } + else + { + prev_heartbeat_time = std::chrono::steady_clock::now(); + sendRequest(xid, HeartbeatRequest()); + } + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + expired = true; + stop = true; + } + + /// TODO drain queue +} + + +void ZooKeeper::receiveThread() +{ + +} + + +void ZooKeeper::sendRequest(XID xid, const Request & request) +{ + /// Excessive copy to calculate length. + WriteBufferFromOwnString buf; + write(xid, buf); + request.write(buf); + write(buf.str()); +} + + +void ZooKeeper::HeartbeatRequest::write(WriteBuffer & out) const +{ + int32_t op_num = 11; + write(op_num); +} + + +} diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h new file mode 100644 index 00000000000..2e1b73d5cd3 --- /dev/null +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -0,0 +1,251 @@ +#pragma once + +#include +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include +#include +#include +#include +#include + + +namespace ZooKeeperImpl +{ + +using namespace DB; + +/** Usage scenario: + * - create an object and issue commands; + * - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap: + * for example, just signal a condvar / fulfull a promise. + * - you also may provide callbacks for watches; they are also invoked in internal thread and must be cheap. + * - whenever you receive SessionExpired exception of method isValid returns false, + * the ZooKeeper instance is no longer usable - you may only destroy it and probably create another. + * - whenever session is expired or ZooKeeper instance is destroying, all callbacks are notified with special event. + * - data for callbacks must be alive when ZooKeeper instance is alive. + */ +class ZooKeeper +{ +public: + using Addresses = std::vector; + + struct ACL + { + int32_t permissions; + String scheme; + String id; + + void write(WriteBuffer & out); + }; + using ACLs = std::vector; + + using WatchCallback = std::function; + + struct Stat + { + int64_t czxid; + int64_t mzxid; + int64_t ctime; + int64_t mtime; + int32_t version; + int32_t cversion; + int32_t aversion; + int64_t ephemeralOwner; + int32_t dataLength; + int32_t numChildren; + int64_t pzxid; + + void read(ReadBuffer & in); + }; + + struct Request + { + virtual ~Request() {}; + virtual int32_t op_num() const = 0; + virtual void write(WriteBuffer & out) const = 0; + }; + + using RequestPtr = std::unique_ptr; + using Requests = std::vector; + + struct Response + { + virtual ~Response() {} + virtual void read(ReadBuffer & in) = 0; + }; + + using ResponsePtr = std::unique_ptr; + using ResponseCallback = std::function; + + struct HeartbeatRequest : Request + { + void write(WriteBuffer & out) const override; + }; + + struct HeartbeatResponse : Response + { + void read(ReadBuffer & in) override; + }; + + struct CreateRequest : Request + { + String path; + String data; + bool is_ephemeral; + bool is_sequential; + ACLs acls; + + void write(WriteBuffer & out) const override; + }; + + struct CreateResponse : Response + { + String path_created; + + void read(ReadBuffer & in) override; + }; + + using CreateCallback = std::function; + + struct RemoveRequest : Request + { + String path; + + void write(WriteBuffer & out) const override; + }; + + struct RemoveResponse : Response + { + void read(ReadBuffer & in) override {} + }; + + struct ExistsRequest : Request + { + static constexpr int32_t op_num = 3; + + String path; + + void write(WriteBuffer & out) const override; + }; + + struct ExistsResponse : Response + { + Stat stat; + + void read(ReadBuffer & in) override; + }; + + struct GetRequest : Request + { + static constexpr int32_t op_num = 4; + + String path; + + void write(WriteBuffer & out) const override; + }; + + struct GetResponse : Response + { + String data; + Stat stat; + + void read(ReadBuffer & in) override; + }; + + using RemoveCallback = std::function; + + /// Connection to addresses is performed in order. If you want, shuffle them manually. + ZooKeeper( + const Addresses & addresses, + const String & root_path, + const String & auth_scheme, + const String & auth_data, + Poco::Timespan session_timeout, + Poco::Timespan connection_timeout); + + ~ZooKeeper(); + + /// If not valid, you can only destroy the object. All other methods will throw exception. + bool isValid() const { return !expired; } + + void create( + const String & path, + const String & data, + bool is_ephemeral, + bool is_sequential, + const ACLs & acls, + CreateCallback callback); + + void remove( + const String & path); + + void exists( + const String & path); + + void get( + const String & path); + + void set( + const String & path, + const String & data); + + void list( + const String & path); + + void check( + const String & path); + + void multi(); + + void close(); + +private: + String root_path; + Poco::Timespan session_timeout; + + Poco::Net::StreamSocket socket; + std::optional in; + std::optional out; + + ConcurrentBoundedQueue requests(1); + + using XID = uint32_t; + using ResponseCallbacks = std::map; + + ResponseCallbacks response_callbacks; + std::mutex response_callbacks_mutex; + + std::thread send_thread; + std::thread receive_thread; + + std::atomic stop {false}; + std::atomic expired {false}; + + void connect(); + void sendHandshake(); + void receiveHandshake(); + void sendAuth(XID xid, const String & auth_scheme, const String & auth_data); + void sendRequest(XID xid, const Request & request); + void receiveEvent(); + + void sendThread(ReadBuffer & in); + void receiveThread(ReadBuffer & in); + + template + void write(const T &); + + template + void read(T &); +}; + +}; From c1c5aa00ca919bd5249a9577db2ceb5d74104610 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Mar 2018 21:14:01 +0300 Subject: [PATCH 19/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 300 +++++++++++++++--- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 137 +++++--- .../src/Common/ZooKeeper/tests/CMakeLists.txt | 3 + .../Common/ZooKeeper/tests/zookeeper_impl.cpp | 26 ++ 4 files changed, 374 insertions(+), 92 deletions(-) create mode 100644 dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7f5bb113623..36326250bd3 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1,20 +1,24 @@ #include #include +#include #include #include -#include +#include #include #include #include +#include + +#include /** ZooKeeper wire protocol. Debugging example: -strace -f -e trace=network -s1000 -x ./clickhouse-zookeeper-cli localhost:2181 +strace -t -f -e trace=network -s1000 -x ./clickhouse-zookeeper-cli localhost:2181 All numbers are in network byte order (big endian). Sizes are 32 bit. Numbers are signed. @@ -104,6 +108,7 @@ namespace ZooKeeperImpl using namespace DB; + /// Assuming we are at little endian. void write(int64_t x, WriteBuffer & out) @@ -120,21 +125,22 @@ void write(int32_t x, WriteBuffer & out) void write(const String & s, WriteBuffer & out) { - write(int32_t(s.size())); + write(int32_t(s.size()), out); out.write(s.data(), s.size()); } -template void write(char s[N], WriteBuffer & out) +template void write(std::array s, WriteBuffer & out) { - write(int32_t(N)); - out.write(s, N); + std::cerr << __PRETTY_FUNCTION__ << "\n"; + write(int32_t(N), out); + out.write(s.data(), N); } template void write(const std::vector & arr, WriteBuffer & out) { - write(int32_t(arr.size())); + write(int32_t(arr.size()), out); for (const auto & elem : arr) - write(elem); + write(elem, out); } void write(const ZooKeeper::ACL & acl, WriteBuffer & out) @@ -142,6 +148,13 @@ void write(const ZooKeeper::ACL & acl, WriteBuffer & out) acl.write(out); } +void ZooKeeper::ACL::write(WriteBuffer & out) const +{ + ZooKeeperImpl::write(permissions, out); + ZooKeeperImpl::write(scheme, out); + ZooKeeperImpl::write(id, out); +} + void read(int64_t & x, ReadBuffer & in) { @@ -159,20 +172,20 @@ void read(String & s, ReadBuffer & in) { static constexpr int32_t max_string_size = 1 << 20; int32_t size = 0; - read(size); + read(size, in); if (size > max_string_size) throw Exception("Too large string size"); /// TODO error code s.resize(size); in.read(&s[0], size); } -template void read(char(&arr)[N], ReadBuffer & in) +template void read(std::array & s, ReadBuffer & in) { int32_t size = 0; - read(size); + read(size, in); if (size != N) throw Exception("Unexpected array size"); /// TODO error code - in.read(arr, N); + in.read(&s[0], N); } void read(ZooKeeper::Stat & stat, ReadBuffer & in) @@ -180,20 +193,45 @@ void read(ZooKeeper::Stat & stat, ReadBuffer & in) stat.read(in); } +void ZooKeeper::Stat::read(ReadBuffer & in) +{ + ZooKeeperImpl::read(czxid, in); + ZooKeeperImpl::read(mzxid, in); + ZooKeeperImpl::read(ctime, in); + ZooKeeperImpl::read(mtime, in); + ZooKeeperImpl::read(version, in); + ZooKeeperImpl::read(cversion, in); + ZooKeeperImpl::read(aversion, in); + ZooKeeperImpl::read(ephemeralOwner, in); + ZooKeeperImpl::read(dataLength, in); + ZooKeeperImpl::read(numChildren, in); + ZooKeeperImpl::read(pzxid, in); +} + template void ZooKeeper::write(const T & x) { - write(x, out); + std::cerr << __PRETTY_FUNCTION__ << "\n"; + ZooKeeperImpl::write(x, *out); } template void ZooKeeper::read(T & x) { - read(x, in); + ZooKeeperImpl::read(x, *in); } +static constexpr int32_t protocol_version = 0; + +//static constexpr ZooKeeper::XID watch_xid = -1; +static constexpr ZooKeeper::XID ping_xid = -2; +//static constexpr ZooKeeper::XID auth_xid = -4; + +static constexpr ZooKeeper::XID close_xid = -3; + + ZooKeeper::~ZooKeeper() { @@ -204,20 +242,29 @@ ZooKeeper::~ZooKeeper() if (receive_thread.joinable()) receive_thread.join(); + + if (!expired) + close(); } ZooKeeper::ZooKeeper( const Addresses & addresses, - const String & root_path, + const String & root_path_, const String & auth_scheme, const String & auth_data, Poco::Timespan session_timeout, Poco::Timespan connection_timeout) - : root_path(root_path), + : root_path(root_path_), session_timeout(session_timeout) { - connect(); + if (!root_path.empty()) + { + if (root_path.back() == '/') + root_path.pop_back(); + } + + connect(addresses, connection_timeout); sendHandshake(); receiveHandshake(); @@ -231,7 +278,8 @@ ZooKeeper::ZooKeeper( void ZooKeeper::connect( - const Addresses & addresses) + const Addresses & addresses, + Poco::Timespan connection_timeout) { static constexpr size_t num_tries = 3; bool connected = false; @@ -262,9 +310,9 @@ void ZooKeeper::connect( if (!connected) throw Exception("All connection tries failed"); /// TODO more info; error code - socket->setReceiveTimeout(session_timeout); - socket->setSendTimeout(session_timeout); - socket->setNoDelay(true); + socket.setReceiveTimeout(session_timeout); + socket.setSendTimeout(session_timeout); + socket.setNoDelay(true); in.emplace(socket); out.emplace(socket); @@ -274,12 +322,11 @@ void ZooKeeper::connect( void ZooKeeper::sendHandshake() { int32_t handshake_length = 44; - int32_t protocol_version = 0; int64_t last_zxid_seen = 0; int32_t timeout = session_timeout.totalMilliseconds(); int64_t session_id = 0; constexpr int32_t passwd_len = 16; - char passwd[passwd_len] {}; + std::array passwd {}; write(handshake_length); write(protocol_version); @@ -288,26 +335,26 @@ void ZooKeeper::sendHandshake() write(session_id); write(passwd); - out.next(); + out->next(); } void ZooKeeper::receiveHandshake() { int32_t handshake_length; - int32_t protocol_version; + int32_t protocol_version_read; int32_t timeout; int64_t session_id; constexpr int32_t passwd_len = 16; - char passwd[passwd_len] {}; + std::array passwd; read(handshake_length); if (handshake_length != 36) throw Exception("Unexpected handshake length received: " + toString(handshake_length)); - read(protocol_version); - if (protocol_version != 0) - throw Exception("Unexpected protocol version: " + toString(protocol_version)); + read(protocol_version_read); + if (protocol_version_read != protocol_version) + throw Exception("Unexpected protocol version: " + toString(protocol_version_read)); read(timeout); if (timeout != session_timeout.totalMilliseconds()) @@ -318,9 +365,18 @@ void ZooKeeper::receiveHandshake() } -void ZooKeeper::sendAuth(XID xid, const String & auth_scheme, const String & auth_data) +/*void ZooKeeper::sendAuth(XID xid, const String & auth_scheme, const String & auth_data) { // TODO +}*/ + + +void ZooKeeper::close() +{ + CloseRequest request; + request.xid = close_xid; + request.write(*out); + expired = true; } @@ -336,20 +392,30 @@ void ZooKeeper::sendThread() ++xid; auto now = std::chrono::steady_clock::now(); - auto next_heartbeat_time = prev_heartbeat_time + std::chrono::duration(session_timeout.totalMilliseconds()); - auto max_wait = next_heartbeat_time > now - ? std::chrono::duration_cast(next_heartbeat_time - now) - : 0; + auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(session_timeout.totalMilliseconds() / 3); + UInt64 max_wait = 0; + if (next_heartbeat_time > now) + max_wait = std::chrono::duration_cast(next_heartbeat_time - now).count(); - RequestPtr request; - if (requests.tryPop(request, max_wait)) + RequestInfo request_info; + if (requests.tryPop(request_info, max_wait)) { - sendRequest(xid, *request); + request_info.request->xid = xid; + + { + std::lock_guard lock(operations_mutex); + operations[xid] = request_info; + } + + request_info.request->write(*out); } else { prev_heartbeat_time = std::chrono::steady_clock::now(); - sendRequest(xid, HeartbeatRequest()); + + HeartbeatRequest request; + request.xid = ping_xid; + request.write(*out); } } } @@ -366,24 +432,168 @@ void ZooKeeper::sendThread() void ZooKeeper::receiveThread() { + try + { + while (!stop) + { + if (!in->poll(session_timeout.totalMicroseconds())) + throw Exception("Nothing is received in session timeout"); + receiveEvent(); + } + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + expired = true; + stop = true; + } } -void ZooKeeper::sendRequest(XID xid, const Request & request) +void ZooKeeper::Request::write(WriteBuffer & out) const { /// Excessive copy to calculate length. WriteBufferFromOwnString buf; - write(xid, buf); - request.write(buf); - write(buf.str()); + ZooKeeperImpl::write(xid, buf); + ZooKeeperImpl::write(getOpNum(), buf); + writeImpl(buf); + ZooKeeperImpl::write(buf.str(), out); + out.next(); } -void ZooKeeper::HeartbeatRequest::write(WriteBuffer & out) const +ZooKeeper::ResponsePtr ZooKeeper::HeartbeatRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::CreateRequest::makeResponse() const { return std::make_shared(); } +//ZooKeeper::ResponsePtr ZooKeeper::RemoveRequest::makeResponse() const { return std::make_shared(); } +//ZooKeeper::ResponsePtr ZooKeeper::ExistsRequest::makeResponse() const { return std::make_shared(); } +//ZooKeeper::ResponsePtr ZooKeeper::GetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { throw Exception("Received response for close request"); } + + +void ZooKeeper::receiveEvent() { - int32_t op_num = 11; - write(op_num); + int32_t length; + int32_t xid; + int64_t zxid; + int32_t err; + + read(length); + size_t count_before_event = in->count(); + read(xid); + read(zxid); + read(err); + + if (xid == ping_xid) + { + /// TODO process err + return; + } + + RequestInfo request_info; + + { + std::lock_guard lock(operations_mutex); + + auto it = operations.find(xid); + if (it == operations.end()) + throw Exception("Received response for unknown xid"); + + request_info = std::move(it->second); + operations.erase(it); + } + + ResponsePtr response = request_info.request->makeResponse(); + + if (err) + response->error = err; + else + response->readImpl(*in); + + int32_t actual_length = in->count() - count_before_event; + if (length != actual_length) + throw Exception("Response length doesn't match"); + + if (request_info.callback) + request_info.callback(*response); +} + + +void ZooKeeper::CreateRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(path, out); + ZooKeeperImpl::write(data, out); + ZooKeeperImpl::write(acls, out); + + int32_t flags = 0; + + if (is_ephemeral) + flags |= 1; + if (is_sequential) + flags |= 2; + + ZooKeeperImpl::write(flags, out); +} + + +void ZooKeeper::CreateResponse::readImpl(ReadBuffer & in) +{ + ZooKeeperImpl::read(path_created, in); +} + + +String ZooKeeper::addRootPath(const String & path) +{ + if (path.empty()) + throw Exception("Path cannot be empty"); + + if (path[0] != '/') + throw Exception("Path must begin with /"); + + if (root_path.empty()) + return path; + + return root_path + path; +} + +String ZooKeeper::removeRootPath(const String & path) +{ + if (root_path.empty()) + return path; + + if (path.size() <= root_path.size()) + throw Exception("Received path is not longer than root_path"); + + return path.substr(root_path.size()); +} + + +void ZooKeeper::create( + const String & path, + const String & data, + bool is_ephemeral, + bool is_sequential, + const ACLs & acls, + CreateCallback callback) +{ + CreateRequest request; + request.path = addRootPath(path); + request.data = data; + request.is_ephemeral = is_ephemeral; + request.is_sequential = is_sequential; + request.acls = acls; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback, this](const Response & response) + { + auto concrete_response = typeid_cast(response); + concrete_response.path_created = removeRootPath(concrete_response.path_created); + callback(concrete_response); + }; + + if (!requests.tryPush(request_info, session_timeout.totalMilliseconds())) + throw Exception("Cannot push request to queue within session timeout"); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 2e1b73d5cd3..28c78fb13c4 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include @@ -13,6 +13,7 @@ #include #include +#include #include #include #include @@ -45,7 +46,7 @@ public: String scheme; String id; - void write(WriteBuffer & out); + void write(WriteBuffer & out) const; }; using ACLs = std::vector; @@ -68,36 +69,56 @@ public: void read(ReadBuffer & in); }; - struct Request - { - virtual ~Request() {}; - virtual int32_t op_num() const = 0; - virtual void write(WriteBuffer & out) const = 0; - }; - - using RequestPtr = std::unique_ptr; - using Requests = std::vector; + using XID = int32_t; + using OpNum = int32_t; struct Response { + int32_t error = 0; virtual ~Response() {} - virtual void read(ReadBuffer & in) = 0; + virtual void readImpl(ReadBuffer &) = 0; }; - using ResponsePtr = std::unique_ptr; + using ResponsePtr = std::shared_ptr; using ResponseCallback = std::function; - struct HeartbeatRequest : Request + struct Request { - void write(WriteBuffer & out) const override; + XID xid; + + virtual ~Request() {}; + virtual OpNum getOpNum() const = 0; + + /// Writes length, xid, op_num, then the rest. + void write(WriteBuffer & out) const; + virtual void writeImpl(WriteBuffer &) const = 0; + + virtual ResponsePtr makeResponse() const = 0; }; - struct HeartbeatResponse : Response + using RequestPtr = std::shared_ptr; + using Requests = std::vector; + + struct HeartbeatRequest final : Request { - void read(ReadBuffer & in) override; + OpNum getOpNum() const override { return 11; } + void writeImpl(WriteBuffer &) const override {} + ResponsePtr makeResponse() const override; }; - struct CreateRequest : Request + struct HeartbeatResponse final : Response + { + void readImpl(ReadBuffer &) override {} + }; + + struct CloseRequest final : Request + { + OpNum getOpNum() const override { return -11; } + void writeImpl(WriteBuffer &) const override {} + ResponsePtr makeResponse() const override; + }; + + struct CreateRequest final : Request { String path; String data; @@ -105,64 +126,72 @@ public: bool is_sequential; ACLs acls; - void write(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 1; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; }; - struct CreateResponse : Response + struct CreateResponse final : Response { String path_created; - void read(ReadBuffer & in) override; + void readImpl(ReadBuffer &) override; }; using CreateCallback = std::function; - struct RemoveRequest : Request + struct RemoveRequest final : Request { String path; - void write(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 2; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; }; - struct RemoveResponse : Response + struct RemoveResponse final : Response { - void read(ReadBuffer & in) override {} + void readImpl(ReadBuffer &) override {} }; - struct ExistsRequest : Request - { - static constexpr int32_t op_num = 3; + using RemoveCallback = std::function; + struct ExistsRequest final : Request + { String path; - void write(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 3; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; }; - struct ExistsResponse : Response + struct ExistsResponse final : Response { Stat stat; - void read(ReadBuffer & in) override; + void readImpl(ReadBuffer &) override; }; - struct GetRequest : Request - { - static constexpr int32_t op_num = 4; + using ExistsCallback = std::function; + struct GetRequest final : Request + { String path; - void write(WriteBuffer & out) const override; + OpNum getOpNum() const override { return 4; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; }; - struct GetResponse : Response + struct GetResponse final : Response { String data; Stat stat; - void read(ReadBuffer & in) override; + void readImpl(ReadBuffer &) override; }; - using RemoveCallback = std::function; + using GetCallback = std::function; /// Connection to addresses is performed in order. If you want, shuffle them manually. ZooKeeper( @@ -217,13 +246,20 @@ private: std::optional in; std::optional out; - ConcurrentBoundedQueue requests(1); + struct RequestInfo + { + RequestPtr request; + ResponseCallback callback; + }; - using XID = uint32_t; - using ResponseCallbacks = std::map; + using RequestsQueue = ConcurrentBoundedQueue; - ResponseCallbacks response_callbacks; - std::mutex response_callbacks_mutex; + RequestsQueue requests{1}; + + using Operations = std::map; + + Operations operations; + std::mutex operations_mutex; std::thread send_thread; std::thread receive_thread; @@ -231,15 +267,22 @@ private: std::atomic stop {false}; std::atomic expired {false}; - void connect(); + String addRootPath(const String &); + String removeRootPath(const String &); + + void connect( + const Addresses & addresses, + Poco::Timespan connection_timeout); + void sendHandshake(); void receiveHandshake(); + void sendAuth(XID xid, const String & auth_scheme, const String & auth_data); - void sendRequest(XID xid, const Request & request); + void receiveEvent(); - void sendThread(ReadBuffer & in); - void receiveThread(ReadBuffer & in); + void sendThread(); + void receiveThread(); template void write(const T &); diff --git a/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt b/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt index a5a035e558b..53bfb75ee12 100644 --- a/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt +++ b/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt @@ -15,3 +15,6 @@ target_link_libraries(zkutil_zookeeper_holder clickhouse_common_zookeeper) add_executable (zk_many_watches_reconnect zk_many_watches_reconnect.cpp) target_link_libraries (zk_many_watches_reconnect clickhouse_common_zookeeper clickhouse_common_config) + +add_executable (zookeeper_impl zookeeper_impl.cpp) +target_link_libraries (zookeeper_impl clickhouse_common_zookeeper) diff --git a/dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp b/dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp new file mode 100644 index 00000000000..51e2760914f --- /dev/null +++ b/dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp @@ -0,0 +1,26 @@ +#include +#include + + +int main() +try +{ + ZooKeeperImpl::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}); + + zookeeper.create("/test", "hello", false, false, {}, [](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) + { + if (response.error) + std::cerr << "Error: " << response.error << "\n"; + else + std::cerr << "Path created: " << response.path_created << "\n"; + }); + + sleep(100); + + return 0; +} +catch (...) +{ + DB::tryLogCurrentException(__PRETTY_FUNCTION__); + return 1; +} From a4e1bd5bc4c8e51062b9ac323d171f933ffb93f6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 18 Mar 2018 21:31:18 +0300 Subject: [PATCH 20/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 54 ++++++++++++++++++- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 52 ++++++++++++++++++ .../Common/ZooKeeper/tests/zookeeper_impl.cpp | 2 +- 3 files changed, 106 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 36326250bd3..cd01013fdd5 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -232,6 +232,41 @@ static constexpr ZooKeeper::XID ping_xid = -2; static constexpr ZooKeeper::XID close_xid = -3; +const char * ZooKeeper::errorMessage(int32_t code) +{ + switch (code) + { + case ZOK: return "Ok"; + case ZSYSTEMERROR: return "System error"; + case ZRUNTIMEINCONSISTENCY: return "Run time inconsistency"; + case ZDATAINCONSISTENCY: return "Data inconsistency"; + case ZCONNECTIONLOSS: return "Connection loss"; + case ZMARSHALLINGERROR: return "Marshalling error"; + case ZUNIMPLEMENTED: return "Unimplemented"; + case ZOPERATIONTIMEOUT: return "Operation timeout"; + case ZBADARGUMENTS: return "Bad arguments"; + case ZINVALIDSTATE: return "Invalid zhandle state"; + case ZAPIERROR: return "API error"; + case ZNONODE: return "No node"; + case ZNOAUTH: return "Not authenticated"; + case ZBADVERSION: return "Bad version"; + case ZNOCHILDRENFOREPHEMERALS: return "No children for ephemerals"; + case ZNODEEXISTS: return "Node exists"; + case ZNOTEMPTY: return "Not empty"; + case ZSESSIONEXPIRED: return "Session expired"; + case ZINVALIDCALLBACK: return "Invalid callback"; + case ZINVALIDACL: return "Invalid ACL"; + case ZAUTHFAILED: return "Authentication failed"; + case ZCLOSING: return "ZooKeeper is closing"; + case ZNOTHING: return "(not error) no server responses to process"; + case ZSESSIONMOVED: return "Session moved to another server, so operation is ignored"; + } + if (code > 0) + return strerror(code); + + return "unknown error"; +} + ZooKeeper::~ZooKeeper() { @@ -264,6 +299,23 @@ ZooKeeper::ZooKeeper( root_path.pop_back(); } + if (auth_scheme.empty()) + { + ACL acl; + acl.permissions = ACL::All; + acl.scheme = "world"; + acl.id = "anyone"; + default_acls.emplace_back(std::move(acl)); + } + else + { + ACL acl; + acl.permissions = ACL::All; + acl.scheme = "auth"; + acl.id = ""; + default_acls.emplace_back(std::move(acl)); + } + connect(addresses, connection_timeout); sendHandshake(); @@ -581,7 +633,7 @@ void ZooKeeper::create( request.data = data; request.is_ephemeral = is_ephemeral; request.is_sequential = is_sequential; - request.acls = acls; + request.acls = acls.empty() ? default_acls : acls; RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 28c78fb13c4..f923906d0e2 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -42,6 +42,13 @@ public: struct ACL { + static constexpr int32_t Read = 1; + static constexpr int32_t Write = 2; + static constexpr int32_t Create = 4; + static constexpr int32_t Delete = 8; + static constexpr int32_t Admin = 16; + static constexpr int32_t All = 0x1F; + int32_t permissions; String scheme; String id; @@ -238,8 +245,53 @@ public: void close(); + + enum ZOO_ERRORS + { + ZOK = 0, /*!< Everything is OK */ + + /** System and server-side errors. + * This is never thrown by the server, it shouldn't be used other than + * to indicate a range. Specifically error codes greater than this + * value, but lesser than {@link #ZAPIERROR}, are system errors. */ + ZSYSTEMERROR = -1, + ZRUNTIMEINCONSISTENCY = -2, /*!< A runtime inconsistency was found */ + ZDATAINCONSISTENCY = -3, /*!< A data inconsistency was found */ + ZCONNECTIONLOSS = -4, /*!< Connection to the server has been lost */ + ZMARSHALLINGERROR = -5, /*!< Error while marshalling or unmarshalling data */ + ZUNIMPLEMENTED = -6, /*!< Operation is unimplemented */ + ZOPERATIONTIMEOUT = -7, /*!< Operation timeout */ + ZBADARGUMENTS = -8, /*!< Invalid arguments */ + ZINVALIDSTATE = -9, /*!< Invliad zhandle state */ + + /** API errors. + * This is never thrown by the server, it shouldn't be used other than + * to indicate a range. Specifically error codes greater than this + * value are API errors (while values less than this indicate a + * {@link #ZSYSTEMERROR}). + */ + ZAPIERROR = -100, + ZNONODE = -101, /*!< Node does not exist */ + ZNOAUTH = -102, /*!< Not authenticated */ + ZBADVERSION = -103, /*!< Version conflict */ + ZNOCHILDRENFOREPHEMERALS = -108, /*!< Ephemeral nodes may not have children */ + ZNODEEXISTS = -110, /*!< The node already exists */ + ZNOTEMPTY = -111, /*!< The node has children */ + ZSESSIONEXPIRED = -112, /*!< The session has been expired by the server */ + ZINVALIDCALLBACK = -113, /*!< Invalid callback specified */ + ZINVALIDACL = -114, /*!< Invalid ACL specified */ + ZAUTHFAILED = -115, /*!< Client authentication failed */ + ZCLOSING = -116, /*!< ZooKeeper is closing */ + ZNOTHING = -117, /*!< (not error) no server responses to process */ + ZSESSIONMOVED = -118 /*! Date: Sun, 18 Mar 2018 23:25:40 +0300 Subject: [PATCH 21/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 356 +++++++++++++++++--- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 173 +++++++--- 2 files changed, 437 insertions(+), 92 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index cd01013fdd5..b365e8bfcfa 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -77,7 +77,7 @@ Client sends requests. For example, create persistent node '/hello' with value ' int32_t request_length \x00\x00\x00\x3a int32_t xid \x5a\xad\x72\x3f Arbitary number. Used for identification of requests/responses. libzookeeper uses unix timestamp for first xid and then autoincrement to that value. -int32_t type \x00\x00\x00\x01 ZOO_CREATE_OP 1 +int32_t op_num \x00\x00\x00\x01 ZOO_CREATE_OP 1 int32_t path_length \x00\x00\x00\x06 path \x2f\x68\x65\x6c\x6c\x6f /hello int32_t data_length \x00\x00\x00\x05 @@ -100,6 +100,38 @@ int64 zxid \x00\x00\x00\x00\x00\x01\x87\x99 int32_t err \x00\x00\x00\x00 string path_created \x00\x00\x00\x06 \x2f\x68\x65\x6c\x6c\x6f /hello - may differ to original path in case of sequential nodes. + + +Client may place a watch in their request. + +For example, client sends "exists" request with watch: + +request length \x00\x00\x00\x12 +xid \x5a\xae\xb2\x0d +op_num \x00\x00\x00\x03 +path \x00\x00\x00\x05 + \x2f\x74\x65\x73\x74 /test +bool watch \x01 + +Server will send response as usual. +And later, server may send special watch event. + +struct WatcherEvent +{ + int32_t type; + int32_t state; + char * path; +}; + +response length \x00\x00\x00\x21 +special watch xid \xff\xff\xff\xff +special watch zxid \xff\xff\xff\xff\xff\xff\xff\xff +err \x00\x00\x00\x00 +type \x00\x00\x00\x02 DELETED_EVENT_DEF 2 +state \x00\x00\x00\x03 CONNECTED_STATE_DEF 3 +path \x00\x00\x00\x05 + \x2f\x74\x65\x73\x74 /test + */ @@ -173,6 +205,8 @@ void read(String & s, ReadBuffer & in) static constexpr int32_t max_string_size = 1 << 20; int32_t size = 0; read(size, in); + if (size < 0) + throw Exception("Negative size"); if (size > max_string_size) throw Exception("Too large string size"); /// TODO error code s.resize(size); @@ -208,6 +242,20 @@ void ZooKeeper::Stat::read(ReadBuffer & in) ZooKeeperImpl::read(pzxid, in); } +template void read(std::vector & arr, ReadBuffer & in) +{ + static constexpr int32_t max_array_size = 1 << 20; + int32_t size = 0; + read(size, in); + if (size < 0) + throw Exception("Negative size"); + if (size > max_array_size) + throw Exception("Too large array size"); /// TODO error code + arr.resize(size); + for (auto & elem : arr) + read(elem, in); +} + template void ZooKeeper::write(const T & x) @@ -225,7 +273,7 @@ void ZooKeeper::read(T & x) static constexpr int32_t protocol_version = 0; -//static constexpr ZooKeeper::XID watch_xid = -1; +static constexpr ZooKeeper::XID watch_xid = -1; static constexpr ZooKeeper::XID ping_xid = -2; //static constexpr ZooKeeper::XID auth_xid = -4; @@ -322,7 +370,7 @@ ZooKeeper::ZooKeeper( receiveHandshake(); if (!auth_scheme.empty()) - sendAuth(-1, auth_scheme, auth_data); + sendAuth(auth_scheme, auth_data); send_thread = std::thread([this] { sendThread(); }); receive_thread = std::thread([this] { receiveThread(); }); @@ -434,15 +482,13 @@ void ZooKeeper::close() void ZooKeeper::sendThread() { - XID xid = 0; /// TODO deal with xid overflow + XID xid = 2; /// TODO deal with xid overflow /// NOTE xid = 1 is reserved for auth request. auto prev_heartbeat_time = std::chrono::steady_clock::now(); try { while (!stop) { - ++xid; - auto now = std::chrono::steady_clock::now(); auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(session_timeout.totalMilliseconds() / 3); UInt64 max_wait = 0; @@ -452,6 +498,7 @@ void ZooKeeper::sendThread() RequestInfo request_info; if (requests.tryPop(request_info, max_wait)) { + request_info.request->addRootPath(root_path); request_info.request->xid = xid; { @@ -459,6 +506,12 @@ void ZooKeeper::sendThread() operations[xid] = request_info; } + if (request_info.watch) + { + std::lock_guard lock(watches_mutex); + watches[request_info.request->getPath()].emplace_back(std::move(request_info.watch)); + } + request_info.request->write(*out); } else @@ -469,6 +522,8 @@ void ZooKeeper::sendThread() request.xid = ping_xid; request.write(*out); } + + ++xid; } } catch (...) @@ -517,11 +572,50 @@ void ZooKeeper::Request::write(WriteBuffer & out) const ZooKeeper::ResponsePtr ZooKeeper::HeartbeatRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::CreateRequest::makeResponse() const { return std::make_shared(); } -//ZooKeeper::ResponsePtr ZooKeeper::RemoveRequest::makeResponse() const { return std::make_shared(); } -//ZooKeeper::ResponsePtr ZooKeeper::ExistsRequest::makeResponse() const { return std::make_shared(); } -//ZooKeeper::ResponsePtr ZooKeeper::GetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::RemoveRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::ExistsRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::GetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::SetRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::ListRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { throw Exception("Received response for close request"); } +void addRootPath(String & path, const String & root_path) +{ + if (path.empty()) + throw Exception("Path cannot be empty"); + + if (path[0] != '/') + throw Exception("Path must begin with /"); + + if (root_path.empty()) + return; + + path = root_path + path; +} + +void removeRootPath(String & path, const String & root_path) +{ + if (root_path.empty()) + return; + + if (path.size() <= root_path.size()) + throw Exception("Received path is not longer than root_path"); + + path = path.substr(root_path.size()); +} + + +void ZooKeeper::CreateRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } +void ZooKeeper::RemoveRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } +void ZooKeeper::ExistsRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } +void ZooKeeper::GetRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } +void ZooKeeper::SetRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } +void ZooKeeper::ListRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } + +void ZooKeeper::CreateResponse::removeRootPath(const String & root_path) { ZooKeeperImpl::removeRootPath(path_created, root_path); } +void ZooKeeper::WatchResponse::removeRootPath(const String & root_path) { ZooKeeperImpl::removeRootPath(path, root_path); } + + void ZooKeeper::receiveEvent() { @@ -538,29 +632,76 @@ void ZooKeeper::receiveEvent() if (xid == ping_xid) { - /// TODO process err + if (err) + throw Exception("Received error in heartbeat response: " + String(errorMessage(err))); return; } - RequestInfo request_info; - + if (xid == watch_xid) { - std::lock_guard lock(operations_mutex); - - auto it = operations.find(xid); - if (it == operations.end()) - throw Exception("Received response for unknown xid"); - - request_info = std::move(it->second); - operations.erase(it); + WatchResponse response; + if (err) + response.error = err; + else + { + response.readImpl(*in); + response.removeRootPath(root_path); + } } - ResponsePtr response = request_info.request->makeResponse(); + RequestInfo request_info; + ResponsePtr response; + + if (xid == ping_xid) + { + if (err) + throw Exception("Received error in heartbeat response: " + String(errorMessage(err))); + + response = std::make_shared(); + } + else if (xid == watch_xid) + { + response = std::make_shared(); + + request_info.callback = [this](const Response & response) + { + const WatchResponse & watch_response = static_cast(response); + + std::lock_guard lock(watches_mutex); + + auto it = watches.find(watch_response.path); + if (it == watches.end()) + throw Exception("Received event for unknown watch"); + + for (auto & callback : it->second) + callback(watch_response); + + watches.erase(it); + }; + } + else + { + { + std::lock_guard lock(operations_mutex); + + auto it = operations.find(xid); + if (it == operations.end()) + throw Exception("Received response for unknown xid"); + + request_info = std::move(it->second); + operations.erase(it); + } + + response = request_info.request->makeResponse(); + } if (err) response->error = err; else + { response->readImpl(*in); + response->removeRootPath(root_path); + } int32_t actual_length = in->count() - count_before_event; if (length != actual_length) @@ -587,36 +728,74 @@ void ZooKeeper::CreateRequest::writeImpl(WriteBuffer & out) const ZooKeeperImpl::write(flags, out); } +void ZooKeeper::RemoveRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(path, out); + ZooKeeperImpl::write(version, out); +} + +void ZooKeeper::ExistsRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(path, out); +} + +void ZooKeeper::GetRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(path, out); +} + +void ZooKeeper::SetRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(path, out); + ZooKeeperImpl::write(data, out); + ZooKeeperImpl::write(version, out); +} + +void ZooKeeper::ListRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(path, out); +} + + +void ZooKeeper::WatchResponse::readImpl(ReadBuffer & in) +{ + ZooKeeperImpl::read(type, in); + ZooKeeperImpl::read(state, in); + ZooKeeperImpl::read(path, in); +} void ZooKeeper::CreateResponse::readImpl(ReadBuffer & in) { ZooKeeperImpl::read(path_created, in); } - -String ZooKeeper::addRootPath(const String & path) +void ZooKeeper::ExistsResponse::readImpl(ReadBuffer & in) { - if (path.empty()) - throw Exception("Path cannot be empty"); - - if (path[0] != '/') - throw Exception("Path must begin with /"); - - if (root_path.empty()) - return path; - - return root_path + path; + ZooKeeperImpl::read(stat, in); } -String ZooKeeper::removeRootPath(const String & path) +void ZooKeeper::GetResponse::readImpl(ReadBuffer & in) { - if (root_path.empty()) - return path; + ZooKeeperImpl::read(data, in); + ZooKeeperImpl::read(stat, in); +} - if (path.size() <= root_path.size()) - throw Exception("Received path is not longer than root_path"); +void ZooKeeper::SetResponse::readImpl(ReadBuffer & in) +{ + ZooKeeperImpl::read(stat, in); +} - return path.substr(root_path.size()); +void ZooKeeper::ListResponse::readImpl(ReadBuffer & in) +{ + ZooKeeperImpl::read(stat, in); + ZooKeeperImpl::read(names, in); +} + + +void ZooKeeper::pushRequest(RequestInfo && info) +{ + if (!requests.tryPush(info, session_timeout.totalMilliseconds())) + throw Exception("Cannot push request to queue within session timeout"); } @@ -629,7 +808,7 @@ void ZooKeeper::create( CreateCallback callback) { CreateRequest request; - request.path = addRootPath(path); + request.path = path; request.data = data; request.is_ephemeral = is_ephemeral; request.is_sequential = is_sequential; @@ -637,15 +816,96 @@ void ZooKeeper::create( RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); - request_info.callback = [callback, this](const Response & response) - { - auto concrete_response = typeid_cast(response); - concrete_response.path_created = removeRootPath(concrete_response.path_created); - callback(concrete_response); - }; + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; - if (!requests.tryPush(request_info, session_timeout.totalMilliseconds())) - throw Exception("Cannot push request to queue within session timeout"); + pushRequest(std::move(request_info)); +} + + +void ZooKeeper::remove( + const String & path, + int32_t version, + RemoveCallback callback) +{ + RemoveRequest request; + request.path = path; + request.version = version; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; + + pushRequest(std::move(request_info)); +} + + +void ZooKeeper::exists( + const String & path, + ExistsCallback callback, + WatchCallback watch) +{ + ExistsRequest request; + request.path = path; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; + request_info.watch = watch; + + pushRequest(std::move(request_info)); +} + + +void ZooKeeper::get( + const String & path, + GetCallback callback, + WatchCallback watch) +{ + GetRequest request; + request.path = path; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; + request_info.watch = watch; + + pushRequest(std::move(request_info)); +} + + +void ZooKeeper::set( + const String & path, + const String & data, + int32_t version, + SetCallback callback) +{ + SetRequest request; + request.path = path; + request.data = data; + request.version = version; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; + + pushRequest(std::move(request_info)); +} + + +void ZooKeeper::list( + const String & path, + ListCallback callback, + WatchCallback watch) +{ + ListRequest request; + request.path = path; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; + request_info.watch = watch; + + pushRequest(std::move(request_info)); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index f923906d0e2..a381b70a270 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -25,6 +25,7 @@ namespace ZooKeeperImpl using namespace DB; + /** Usage scenario: * - create an object and issue commands; * - you provide callbacks for your commands; callbacks are invoked in internal thread and must be cheap: @@ -57,8 +58,6 @@ public: }; using ACLs = std::vector; - using WatchCallback = std::function; - struct Stat { int64_t czxid; @@ -84,6 +83,8 @@ public: int32_t error = 0; virtual ~Response() {} virtual void readImpl(ReadBuffer &) = 0; + + virtual void removeRootPath(const String & /* root_path */) {} }; using ResponsePtr = std::shared_ptr; @@ -101,6 +102,9 @@ public: virtual void writeImpl(WriteBuffer &) const = 0; virtual ResponsePtr makeResponse() const = 0; + + virtual void addRootPath(const String & /* root_path */) {}; + virtual String getPath() const = 0; }; using RequestPtr = std::shared_ptr; @@ -111,6 +115,7 @@ public: OpNum getOpNum() const override { return 11; } void writeImpl(WriteBuffer &) const override {} ResponsePtr makeResponse() const override; + String getPath() const override { return {}; } }; struct HeartbeatResponse final : Response @@ -118,11 +123,24 @@ public: void readImpl(ReadBuffer &) override {} }; + struct WatchResponse final : Response + { + int32_t type; + int32_t state; + String path; + + void readImpl(ReadBuffer &) override; + void removeRootPath(const String & root_path) override; + }; + + using WatchCallback = std::function; + struct CloseRequest final : Request { OpNum getOpNum() const override { return -11; } void writeImpl(WriteBuffer &) const override {} ResponsePtr makeResponse() const override; + String getPath() const override { return {}; } }; struct CreateRequest final : Request @@ -136,6 +154,8 @@ public: OpNum getOpNum() const override { return 1; } void writeImpl(WriteBuffer &) const override; ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } }; struct CreateResponse final : Response @@ -143,6 +163,7 @@ public: String path_created; void readImpl(ReadBuffer &) override; + void removeRootPath(const String & root_path) override; }; using CreateCallback = std::function; @@ -150,10 +171,13 @@ public: struct RemoveRequest final : Request { String path; + int32_t version; OpNum getOpNum() const override { return 2; } void writeImpl(WriteBuffer &) const override; ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } }; struct RemoveResponse final : Response @@ -170,6 +194,8 @@ public: OpNum getOpNum() const override { return 3; } void writeImpl(WriteBuffer &) const override; ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } }; struct ExistsResponse final : Response @@ -188,6 +214,8 @@ public: OpNum getOpNum() const override { return 4; } void writeImpl(WriteBuffer &) const override; ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } }; struct GetResponse final : Response @@ -200,6 +228,49 @@ public: using GetCallback = std::function; + struct SetRequest final : Request + { + String path; + String data; + int32_t version; + + OpNum getOpNum() const override { return 5; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } + }; + + struct SetResponse final : Response + { + Stat stat; + + void readImpl(ReadBuffer &) override; + }; + + using SetCallback = std::function; + + struct ListRequest final : Request + { + String path; + + OpNum getOpNum() const override { return 12; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } + }; + + struct ListResponse final : Response + { + Stat stat; + std::vector names; + + void readImpl(ReadBuffer &) override; + }; + + using ListCallback = std::function; + /// Connection to addresses is performed in order. If you want, shuffle them manually. ZooKeeper( const Addresses & addresses, @@ -223,23 +294,30 @@ public: CreateCallback callback); void remove( - const String & path); + const String & path, + int32_t version, + RemoveCallback callback); void exists( - const String & path); + const String & path, + ExistsCallback callback, + WatchCallback watch); void get( - const String & path); + const String & path, + GetCallback callback, + WatchCallback watch); void set( const String & path, - const String & data); + const String & data, + int32_t version, + SetCallback callback); void list( - const String & path); - - void check( - const String & path); + const String & path, + ListCallback callback, + WatchCallback watch); void multi(); @@ -248,42 +326,44 @@ public: enum ZOO_ERRORS { - ZOK = 0, /*!< Everything is OK */ + ZOK = 0, /** System and server-side errors. - * This is never thrown by the server, it shouldn't be used other than - * to indicate a range. Specifically error codes greater than this - * value, but lesser than {@link #ZAPIERROR}, are system errors. */ + * This is never thrown by the server, it shouldn't be used other than + * to indicate a range. Specifically error codes greater than this + * value, but lesser than ZAPIERROR, are system errors. + */ ZSYSTEMERROR = -1, - ZRUNTIMEINCONSISTENCY = -2, /*!< A runtime inconsistency was found */ - ZDATAINCONSISTENCY = -3, /*!< A data inconsistency was found */ - ZCONNECTIONLOSS = -4, /*!< Connection to the server has been lost */ - ZMARSHALLINGERROR = -5, /*!< Error while marshalling or unmarshalling data */ - ZUNIMPLEMENTED = -6, /*!< Operation is unimplemented */ - ZOPERATIONTIMEOUT = -7, /*!< Operation timeout */ - ZBADARGUMENTS = -8, /*!< Invalid arguments */ - ZINVALIDSTATE = -9, /*!< Invliad zhandle state */ + + ZRUNTIMEINCONSISTENCY = -2, /// A runtime inconsistency was found + ZDATAINCONSISTENCY = -3, /// A data inconsistency was found + ZCONNECTIONLOSS = -4, /// Connection to the server has been lost + ZMARSHALLINGERROR = -5, /// Error while marshalling or unmarshalling data + ZUNIMPLEMENTED = -6, /// Operation is unimplemented + ZOPERATIONTIMEOUT = -7, /// Operation timeout + ZBADARGUMENTS = -8, /// Invalid arguments + ZINVALIDSTATE = -9, /// Invliad zhandle state /** API errors. - * This is never thrown by the server, it shouldn't be used other than - * to indicate a range. Specifically error codes greater than this - * value are API errors (while values less than this indicate a - * {@link #ZSYSTEMERROR}). - */ + * This is never thrown by the server, it shouldn't be used other than + * to indicate a range. Specifically error codes greater than this + * value are API errors. + */ ZAPIERROR = -100, - ZNONODE = -101, /*!< Node does not exist */ - ZNOAUTH = -102, /*!< Not authenticated */ - ZBADVERSION = -103, /*!< Version conflict */ - ZNOCHILDRENFOREPHEMERALS = -108, /*!< Ephemeral nodes may not have children */ - ZNODEEXISTS = -110, /*!< The node already exists */ - ZNOTEMPTY = -111, /*!< The node has children */ - ZSESSIONEXPIRED = -112, /*!< The session has been expired by the server */ - ZINVALIDCALLBACK = -113, /*!< Invalid callback specified */ - ZINVALIDACL = -114, /*!< Invalid ACL specified */ - ZAUTHFAILED = -115, /*!< Client authentication failed */ - ZCLOSING = -116, /*!< ZooKeeper is closing */ - ZNOTHING = -117, /*!< (not error) no server responses to process */ - ZSESSIONMOVED = -118 /*!; RequestsQueue requests{1}; + void pushRequest(RequestInfo && info); using Operations = std::map; Operations operations; std::mutex operations_mutex; + using WatchCallbacks = std::vector; + using Watches = std::map; + + Watches watches; + std::mutex watches_mutex; + std::thread send_thread; std::thread receive_thread; std::atomic stop {false}; std::atomic expired {false}; - String addRootPath(const String &); - String removeRootPath(const String &); - void connect( const Addresses & addresses, Poco::Timespan connection_timeout); @@ -329,7 +414,7 @@ private: void sendHandshake(); void receiveHandshake(); - void sendAuth(XID xid, const String & auth_scheme, const String & auth_data); + void sendAuth(const String & auth_scheme, const String & auth_data); void receiveEvent(); From 347a83029a11a4bd3645a2ee1b47cdd939a64866 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 00:53:56 +0300 Subject: [PATCH 22/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 225 ++++++++++++++++++ dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 55 ++++- .../ZooKeeper/tests/zkutil_test_commands.cpp | 66 ++--- 3 files changed, 315 insertions(+), 31 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index b365e8bfcfa..f332f32c3da 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -132,6 +132,97 @@ state \x00\x00\x00\x03 CONNECTED_STATE_DEF 3 path \x00\x00\x00\x05 \x2f\x74\x65\x73\x74 /test + +Example of multi request: + +request length \x00\x00\x00\x82 130 +xid \x5a\xae\xd6\x16 +op_num \x00\x00\x00\x0e 14 + +for every command: + + int32_t type; \x00\x00\x00\x01 create + bool done; \x00 false + int32_t err; \xff\xff\xff\xff -1 + + path \x00\x00\x00\x05 + \x2f\x74\x65\x73\x74 /test + data \x00\x00\x00\x06 + \x6d\x75\x6c\x74\x69\x31 multi1 + acl \x00\x00\x00\x01 + \x00\x00\x00\x1f + \x00\x00\x00\x05 + \x77\x6f\x72\x6c\x64 world + \x00\x00\x00\x06 + \x61\x6e\x79\x6f\x6e\x65 anyone + flags \x00\x00\x00\x00 + + int32_t type; \x00\x00\x00\x05 set + bool done \x00 false + int32_t err; \xff\xff\xff\xff -1 + + path \x00\x00\x00\x05 + \x2f\x74\x65\x73\x74 + data \x00\x00\x00\x06 + \x6d\x75\x6c\x74\x69\x32 multi2 + version \xff\xff\xff\xff + + int32_t type \x00\x00\x00\x02 remove + bool done \x00 + int32_t err \xff\xff\xff\xff -1 + + path \x00\x00\x00\x05 + \x2f\x74\x65\x73\x74 + version \xff\xff\xff\xff + +after commands: + + int32_t type \xff\xff\xff\xff -1 + bool done \x01 true + int32_t err \xff\xff\xff\xff + +Example of multi response: + +response length \x00\x00\x00\x81 129 +xid \x5a\xae\xd6\x16 +zxid \x00\x00\x00\x00\x00\x01\x87\xe1 +err \x00\x00\x00\x00 + +in a loop: + + type \x00\x00\x00\x01 create + done \x00 + err \x00\x00\x00\x00 + + path_created \x00\x00\x00\x05 + \x2f\x74\x65\x73\x74 + + type \x00\x00\x00\x05 set + done \x00 + err \x00\x00\x00\x00 + + stat \x00\x00\x00\x00\x00\x01\x87\xe1 + \x00\x00\x00\x00\x00\x01\x87\xe1 + \x00\x00\x01\x62\x3a\xf4\x35\x0c + \x00\x00\x01\x62\x3a\xf4\x35\x0c + \x00\x00\x00\x01 + \x00\x00\x00\x00 + \x00\x00\x00\x00 + \x00\x00\x00\x00\x00\x00\x00\x00 + \x00\x00\x00\x06 + \x00\x00\x00\x00 + \x00\x00\x00\x00\x00\x01\x87\xe1 + + type \x00\x00\x00\x02 remove + done \x00 + err \x00\x00\x00\x00 + +after: + + type \xff\xff\xff\xff + done \x01 + err \xff\xff\xff\xff + */ @@ -155,6 +246,11 @@ void write(int32_t x, WriteBuffer & out) writeBinary(x, out); } +void write(bool x, WriteBuffer & out) +{ + writeBinary(x, out); +} + void write(const String & s, WriteBuffer & out) { write(int32_t(s.size()), out); @@ -200,6 +296,11 @@ void read(int32_t & x, ReadBuffer & in) x = __builtin_bswap32(x); } +void read(bool & x, ReadBuffer & in) +{ + readBinary(x, in); +} + void read(String & s, ReadBuffer & in) { static constexpr int32_t max_string_size = 1 << 20; @@ -508,6 +609,7 @@ void ZooKeeper::sendThread() if (request_info.watch) { + request_info.request->has_watch = true; std::lock_guard lock(watches_mutex); watches[request_info.request->getPath()].emplace_back(std::move(request_info.watch)); } @@ -577,6 +679,8 @@ ZooKeeper::ResponsePtr ZooKeeper::ExistsRequest::makeResponse() const { return s ZooKeeper::ResponsePtr ZooKeeper::GetRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::SetRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::ListRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::CheckRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::MultiRequest::makeResponse() const { return std::make_shared(requests); } ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { throw Exception("Received response for close request"); } void addRootPath(String & path, const String & root_path) @@ -611,10 +715,22 @@ void ZooKeeper::ExistsRequest::addRootPath(const String & root_path) { ZooKeeper void ZooKeeper::GetRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } void ZooKeeper::SetRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } void ZooKeeper::ListRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } +void ZooKeeper::CheckRequest::addRootPath(const String & root_path) { ZooKeeperImpl::addRootPath(path, root_path); } + +void ZooKeeper::MultiRequest::addRootPath(const String & root_path) +{ + for (auto & request : requests) + request->addRootPath(root_path); +} void ZooKeeper::CreateResponse::removeRootPath(const String & root_path) { ZooKeeperImpl::removeRootPath(path_created, root_path); } void ZooKeeper::WatchResponse::removeRootPath(const String & root_path) { ZooKeeperImpl::removeRootPath(path, root_path); } +void ZooKeeper::MultiResponse::removeRootPath(const String & root_path) +{ + for (auto & response : responses) + response->removeRootPath(root_path); +} void ZooKeeper::receiveEvent() @@ -737,11 +853,13 @@ void ZooKeeper::RemoveRequest::writeImpl(WriteBuffer & out) const void ZooKeeper::ExistsRequest::writeImpl(WriteBuffer & out) const { ZooKeeperImpl::write(path, out); + ZooKeeperImpl::write(has_watch, out); } void ZooKeeper::GetRequest::writeImpl(WriteBuffer & out) const { ZooKeeperImpl::write(path, out); + ZooKeeperImpl::write(has_watch, out); } void ZooKeeper::SetRequest::writeImpl(WriteBuffer & out) const @@ -754,6 +872,36 @@ void ZooKeeper::SetRequest::writeImpl(WriteBuffer & out) const void ZooKeeper::ListRequest::writeImpl(WriteBuffer & out) const { ZooKeeperImpl::write(path, out); + ZooKeeperImpl::write(has_watch, out); +} + +void ZooKeeper::CheckRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(path, out); + ZooKeeperImpl::write(version, out); +} + +void ZooKeeper::MultiRequest::writeImpl(WriteBuffer & out) const +{ + for (const auto & request : requests) + { + bool done = false; + int32_t error = 0; + + ZooKeeperImpl::write(request->getOpNum(), out); + ZooKeeperImpl::write(done, out); + ZooKeeperImpl::write(error, out); + + request->writeImpl(out); + } + + OpNum op_num = -1; + bool done = true; + int32_t error = -1; + + ZooKeeperImpl::write(op_num, out); + ZooKeeperImpl::write(done, out); + ZooKeeperImpl::write(error, out); } @@ -791,6 +939,51 @@ void ZooKeeper::ListResponse::readImpl(ReadBuffer & in) ZooKeeperImpl::read(names, in); } +ZooKeeper::MultiResponse::MultiResponse(const Requests & requests) +{ + responses.reserve(requests.size()); + + for (const auto & request : requests) + responses.emplace_back(request->makeResponse()); +} + +void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in) +{ + for (const auto & response : responses) + { + OpNum op_num; + bool done; + int32_t error; + + ZooKeeperImpl::read(op_num, in); + ZooKeeperImpl::read(done, in); + ZooKeeperImpl::read(error, in); + + if (done) + throw Exception("Not enough results received for multi transaction"); + + if (error) + response->error = error; + else + response->readImpl(in); + } + + OpNum op_num; + bool done; + int32_t error; + + ZooKeeperImpl::read(op_num, in); + ZooKeeperImpl::read(done, in); + ZooKeeperImpl::read(error, in); + + if (!done) + throw Exception("Too many results received for multi transaction"); + if (op_num != -1) + throw Exception("Unexpected op_num received at the end of results for multi transaction"); + if (error != -1) + throw Exception("Unexpected error value received at the end of results for multi transaction"); +} + void ZooKeeper::pushRequest(RequestInfo && info) { @@ -909,4 +1102,36 @@ void ZooKeeper::list( } +void ZooKeeper::check( + const String & path, + int32_t version, + CheckCallback callback) +{ + CheckRequest request; + request.path = path; + request.version = version; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; + + pushRequest(std::move(request_info)); +} + + +void ZooKeeper::multi( + const Requests & requests, + MultiCallback callback) +{ + MultiRequest request; + request.requests = requests; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; + + pushRequest(std::move(request_info)); +} + + } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index a381b70a270..0811dae2b59 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -78,6 +78,7 @@ public: using XID = int32_t; using OpNum = int32_t; + struct Response { int32_t error = 0; @@ -88,11 +89,13 @@ public: }; using ResponsePtr = std::shared_ptr; + using Responses = std::vector; using ResponseCallback = std::function; struct Request { XID xid; + bool has_watch = false; virtual ~Request() {}; virtual OpNum getOpNum() const = 0; @@ -271,6 +274,49 @@ public: using ListCallback = std::function; + struct CheckRequest final : Request + { + String path; + int32_t version; + + OpNum getOpNum() const override { return 13; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return path; } + }; + + struct CheckResponse final : Response + { + void readImpl(ReadBuffer &) override {}; + }; + + using CheckCallback = std::function; + + struct MultiRequest final : Request + { + Requests requests; + + OpNum getOpNum() const override { return 14; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; + void addRootPath(const String & root_path) override; + String getPath() const override { return {}; } + }; + + struct MultiResponse final : Response + { + Responses responses; + + MultiResponse(const Requests & requests); + + void readImpl(ReadBuffer &) override; + void removeRootPath(const String & root_path) override; + }; + + using MultiCallback = std::function; + + /// Connection to addresses is performed in order. If you want, shuffle them manually. ZooKeeper( const Addresses & addresses, @@ -319,7 +365,14 @@ public: ListCallback callback, WatchCallback watch); - void multi(); + void check( + const String & path, + int32_t version, + CheckCallback callback); + + void multi( + const Requests & requests, + MultiCallback callback); void close(); diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp index 3c9aed5113d..d7fa1d3bf42 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp @@ -4,38 +4,44 @@ using namespace zkutil; -int main() +int main(int argc, char ** argv) +try { - try + if (argc < 2) { - ZooKeeper zk("mtfilter01t:2181,metrika-test:2181,mtweb01t:2181", "", 5000); - Strings children; - - std::cout << "create path" << std::endl; - zk.create("/test", "old", zkutil::CreateMode::Persistent); - zkutil::Stat stat; - zkutil::EventPtr watch = std::make_shared(); - - std::cout << "get path" << std::endl; - zk.get("/test", &stat, watch); - std::cout << "set path" << std::endl; - zk.set("/test", "new"); - watch->wait(); - std::cout << "watch happened" << std::endl; - std::cout << "remove path" << std::endl; - zk.remove("/test"); - - Ops ops; - ops.emplace_back(std::make_unique("/test", "multi1", zk.getDefaultACL(), CreateMode::Persistent)); - ops.emplace_back(std::make_unique("/test", "multi2", -1)); - ops.emplace_back(std::make_unique("/test", -1)); - std::cout << "multi" << std::endl; - OpResultsPtr res = zk.multi(ops); - std::cout << "path created: " << dynamic_cast(*ops[0]).getPathCreated() << std::endl; - } - catch (KeeperException & e) - { - std::cerr << "KeeperException " << e.what() << " " << e.message() << std::endl; + std::cerr << "Usage: ./zkutil_test_commands host:port,host:port...\n"; + return 1; } + + ZooKeeper zk(argv[1], "", 5000); + Strings children; + + std::cout << "create path" << std::endl; + zk.create("/test", "old", zkutil::CreateMode::Persistent); + zkutil::Stat stat; + zkutil::EventPtr watch = std::make_shared(); + + std::cout << "get path" << std::endl; + zk.get("/test", &stat, watch); + std::cout << "set path" << std::endl; + zk.set("/test", "new"); + watch->wait(); + std::cout << "watch happened" << std::endl; + std::cout << "remove path" << std::endl; + zk.remove("/test"); + + Ops ops; + ops.emplace_back(std::make_unique("/test", "multi1", zk.getDefaultACL(), CreateMode::Persistent)); + ops.emplace_back(std::make_unique("/test", "multi2", -1)); + ops.emplace_back(std::make_unique("/test", -1)); + std::cout << "multi" << std::endl; + OpResultsPtr res = zk.multi(ops); + std::cout << "path created: " << dynamic_cast(*ops[0]).getPathCreated() << std::endl; + return 0; } +catch (KeeperException & e) +{ + std::cerr << "KeeperException " << e.what() << " " << e.message() << std::endl; + return 1; +} From 0d11b75defe7dd083b46ff2cd0b8bd795bda6339 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 02:11:57 +0300 Subject: [PATCH 23/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 38 ++--- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 6 +- .../src/Common/ZooKeeper/tests/CMakeLists.txt | 3 + .../tests/zkutil_test_commands_new_lib.cpp | 130 ++++++++++++++++++ 4 files changed, 151 insertions(+), 26 deletions(-) create mode 100644 dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index f332f32c3da..c13f38b895c 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -259,7 +259,6 @@ void write(const String & s, WriteBuffer & out) template void write(std::array s, WriteBuffer & out) { - std::cerr << __PRETTY_FUNCTION__ << "\n"; write(int32_t(N), out); out.write(s.data(), N); } @@ -361,7 +360,6 @@ template void read(std::vector & arr, ReadBuffer & in) template void ZooKeeper::write(const T & x) { - std::cerr << __PRETTY_FUNCTION__ << "\n"; ZooKeeperImpl::write(x, *out); } @@ -746,25 +744,6 @@ void ZooKeeper::receiveEvent() read(zxid); read(err); - if (xid == ping_xid) - { - if (err) - throw Exception("Received error in heartbeat response: " + String(errorMessage(err))); - return; - } - - if (xid == watch_xid) - { - WatchResponse response; - if (err) - response.error = err; - else - { - response.readImpl(*in); - response.removeRootPath(root_path); - } - } - RequestInfo request_info; ResponsePtr response; @@ -774,6 +753,8 @@ void ZooKeeper::receiveEvent() throw Exception("Received error in heartbeat response: " + String(errorMessage(err))); response = std::make_shared(); + + std::cerr << "Received heartbeat\n"; } else if (xid == watch_xid) { @@ -794,6 +775,8 @@ void ZooKeeper::receiveEvent() watches.erase(it); }; + + std::cerr << "Received watch\n"; } else { @@ -808,6 +791,8 @@ void ZooKeeper::receiveEvent() operations.erase(it); } + std::cerr << "Received response: " << request_info.request->getOpNum() << "\n"; + response = request_info.request->makeResponse(); } @@ -821,7 +806,7 @@ void ZooKeeper::receiveEvent() int32_t actual_length = in->count() - count_before_event; if (length != actual_length) - throw Exception("Response length doesn't match"); + throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length)); if (request_info.callback) request_info.callback(*response); @@ -886,7 +871,7 @@ void ZooKeeper::MultiRequest::writeImpl(WriteBuffer & out) const for (const auto & request : requests) { bool done = false; - int32_t error = 0; + int32_t error = -1; ZooKeeperImpl::write(request->getOpNum(), out); ZooKeeperImpl::write(done, out); @@ -959,6 +944,8 @@ void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in) ZooKeeperImpl::read(done, in); ZooKeeperImpl::read(error, in); + std::cerr << "Received result for multi: " << op_num << "\n"; + if (done) throw Exception("Not enough results received for multi transaction"); @@ -1126,6 +1113,11 @@ void ZooKeeper::multi( MultiRequest request; request.requests = requests; + for (auto & elem : request.requests) + if (CreateRequest * create = typeid_cast(elem.get())) + if (create->acls.empty()) + create->acls = default_acls; + RequestInfo request_info; request_info.request = std::make_shared(std::move(request)); request_info.callback = [callback](const Response & response) { callback(typeid_cast(response)); }; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 0811dae2b59..a7129351df2 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -174,7 +174,7 @@ public: struct RemoveRequest final : Request { String path; - int32_t version; + int32_t version = -1; OpNum getOpNum() const override { return 2; } void writeImpl(WriteBuffer &) const override; @@ -235,7 +235,7 @@ public: { String path; String data; - int32_t version; + int32_t version = -1; OpNum getOpNum() const override { return 5; } void writeImpl(WriteBuffer &) const override; @@ -277,7 +277,7 @@ public: struct CheckRequest final : Request { String path; - int32_t version; + int32_t version = -1; OpNum getOpNum() const override { return 13; } void writeImpl(WriteBuffer &) const override; diff --git a/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt b/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt index 53bfb75ee12..a24948240f6 100644 --- a/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt +++ b/dbms/src/Common/ZooKeeper/tests/CMakeLists.txt @@ -1,6 +1,9 @@ add_executable(zkutil_test_commands zkutil_test_commands.cpp) target_link_libraries(zkutil_test_commands clickhouse_common_zookeeper) +add_executable(zkutil_test_commands_new_lib zkutil_test_commands_new_lib.cpp) +target_link_libraries(zkutil_test_commands_new_lib clickhouse_common_zookeeper) + add_executable(zkutil_test_lock zkutil_test_lock.cpp) target_link_libraries(zkutil_test_lock clickhouse_common_zookeeper) diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp new file mode 100644 index 00000000000..3bb3f578fa8 --- /dev/null +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp @@ -0,0 +1,130 @@ +#include +#include +#include +#include +#include +#include + + +using namespace ZooKeeperImpl; + + +int main(int argc, char ** argv) +try +{ + if (argc < 2) + { + std::cerr << "Usage: ./zkutil_test_commands_new_lib host:port,host:port...\n"; + return 1; + } + + Poco::AutoPtr channel(new Poco::ConsoleChannel(std::cerr)); + Poco::Logger::root().setChannel(channel); + Poco::Logger::root().setLevel("trace"); + + std::string addresses_arg = argv[1]; + std::vector addresses_strings; + boost::split(addresses_strings, addresses_arg, boost::is_any_of(",")); + ZooKeeper::Addresses addresses; + addresses.reserve(addresses_strings.size()); + for (const auto & address_string : addresses_strings) + addresses.emplace_back(address_string); + + ZooKeeper zk(addresses, {}, {}, {}, {5, 0}, {0, 50000}); + + Strings children; + + std::cout << "create path" << '\n'; + + zk.create("/test", "old", false, false, {}, [](const ZooKeeper::CreateResponse & response) + { + if (response.error) + std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Created path: " << response.path_created << '\n'; + }); + + std::cout << "get path" << '\n'; + + zk.get("/test", + [](const ZooKeeper::GetResponse & response) + { + if (response.error) + std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Value: " << response.data << '\n'; + }, +/* [](const ZooKeeper::WatchResponse & response) + { + if (response.error) + std::cerr << "Watch, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Watch, path: " << response.path << ", type: " << response.type << '\n'; + }*/ {}); + + std::cout << "set path" << '\n'; + + zk.set("/test", "new", -1, [](const ZooKeeper::SetResponse & response) + { + if (response.error) + std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Set\n"; + }); + + std::cout << "remove path" << '\n'; + + zk.remove("/test", -1, [](const ZooKeeper::RemoveResponse & response) + { + if (response.error) + std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Removed\n"; + }); + + std::cout << "multi" << '\n'; + + ZooKeeper::Requests ops; + + { + ZooKeeper::CreateRequest create_request; + create_request.path = "/test"; + create_request.data = "multi1"; + ops.emplace_back(std::make_shared(std::move(create_request))); + } + + { + ZooKeeper::SetRequest set_request; + set_request.path = "/test"; + set_request.data = "multi2"; + ops.emplace_back(std::make_shared(std::move(set_request))); + } + + { + ZooKeeper::RemoveRequest remove_request; + remove_request.path = "/test"; + ops.emplace_back(std::make_shared(std::move(remove_request))); + } + + zk.multi(ops, [](const ZooKeeper::MultiResponse & response) + { + if (response.error) + std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + { + for (const auto & elem : response.responses) + if (elem->error) + std::cerr << "Error (elem) " << elem->error << ": " << ZooKeeper::errorMessage(elem->error) << '\n'; + + std::cerr << "Created path: " << typeid_cast(*response.responses[0]).path_created << '\n'; + } + }); + + sleep(5); + return 0; +} +catch (...) +{ + std::cerr << DB::getCurrentExceptionMessage(__PRETTY_FUNCTION__) << '\n'; + return 1; +} From 5146ab8acc4236736ee5c1441b9044a18252dc82 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 03:32:59 +0300 Subject: [PATCH 24/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 148 +++++++++++++++----- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 77 +++++++--- 2 files changed, 170 insertions(+), 55 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index c13f38b895c..494ea3ce77d 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -374,9 +374,9 @@ static constexpr int32_t protocol_version = 0; static constexpr ZooKeeper::XID watch_xid = -1; static constexpr ZooKeeper::XID ping_xid = -2; -//static constexpr ZooKeeper::XID auth_xid = -4; +static constexpr ZooKeeper::XID auth_xid = -4; -static constexpr ZooKeeper::XID close_xid = -3; +static constexpr ZooKeeper::XID close_xid = 0x7FFFFFFF; const char * ZooKeeper::errorMessage(int32_t code) @@ -417,16 +417,25 @@ const char * ZooKeeper::errorMessage(int32_t code) ZooKeeper::~ZooKeeper() { - stop = true; + try + { + stop = true; - if (send_thread.joinable()) - send_thread.join(); + if (send_thread.joinable()) + send_thread.join(); - if (receive_thread.joinable()) - receive_thread.join(); + if (receive_thread.joinable()) + receive_thread.join(); - if (!expired) - close(); + if (!expired) + close(); + + finalize(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } } @@ -564,10 +573,35 @@ void ZooKeeper::receiveHandshake() } -/*void ZooKeeper::sendAuth(XID xid, const String & auth_scheme, const String & auth_data) +void ZooKeeper::sendAuth(const String & scheme, const String & data) { - // TODO -}*/ + AuthRequest request; + request.scheme = scheme; + request.data = data; + request.xid = auth_xid; + request.write(*out); + + int32_t length; + int32_t xid; + int64_t zxid; + int32_t err; + + read(length); + size_t count_before_event = in->count(); + read(xid); + read(zxid); + read(err); + + if (xid != auth_xid) + throw Exception("Unexpected event recievent in reply to auth request: " + toString(xid)); + + int32_t actual_length = in->count() - count_before_event; + if (length != actual_length) + throw Exception("Response length doesn't match. Expected: " + toString(length) + ", actual: " + toString(actual_length)); + + if (err) + throw Exception("Error received in reply to auth request. Code: " + toString(err) + ". Message: " + String(errorMessage(err))); +} void ZooKeeper::close() @@ -581,7 +615,6 @@ void ZooKeeper::close() void ZooKeeper::sendThread() { - XID xid = 2; /// TODO deal with xid overflow /// NOTE xid = 1 is reserved for auth request. auto prev_heartbeat_time = std::chrono::steady_clock::now(); try @@ -594,25 +627,10 @@ void ZooKeeper::sendThread() if (next_heartbeat_time > now) max_wait = std::chrono::duration_cast(next_heartbeat_time - now).count(); - RequestInfo request_info; - if (requests.tryPop(request_info, max_wait)) + RequestPtr request; + if (requests.tryPop(request, max_wait)) { - request_info.request->addRootPath(root_path); - request_info.request->xid = xid; - - { - std::lock_guard lock(operations_mutex); - operations[xid] = request_info; - } - - if (request_info.watch) - { - request_info.request->has_watch = true; - std::lock_guard lock(watches_mutex); - watches[request_info.request->getPath()].emplace_back(std::move(request_info.watch)); - } - - request_info.request->write(*out); + request->write(*out); } else { @@ -622,8 +640,6 @@ void ZooKeeper::sendThread() request.xid = ping_xid; request.write(*out); } - - ++xid; } } catch (...) @@ -671,6 +687,7 @@ void ZooKeeper::Request::write(WriteBuffer & out) const ZooKeeper::ResponsePtr ZooKeeper::HeartbeatRequest::makeResponse() const { return std::make_shared(); } +ZooKeeper::ResponsePtr ZooKeeper::AuthRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::CreateRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::RemoveRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::ExistsRequest::makeResponse() const { return std::make_shared(); } @@ -771,7 +788,8 @@ void ZooKeeper::receiveEvent() throw Exception("Received event for unknown watch"); for (auto & callback : it->second) - callback(watch_response); + if (callback) + callback(watch_response); watches.erase(it); }; @@ -813,6 +831,49 @@ void ZooKeeper::receiveEvent() } +void ZooKeeper::finalize() +{ + { + std::lock_guard lock(operations_mutex); + + for (auto & op : operations) + { + RequestInfo & request_info = op.second; + ResponsePtr response = request_info.request->makeResponse(); + response->error = ZCONNECTIONLOSS; + if (request_info.callback) + request_info.callback(*response); + } + + operations.clear(); + } + + { + std::lock_guard lock(watches_mutex); + + for (auto & path_watches : watches) + { + WatchResponse response; + response.type = SESSION; + response.state = EXPIRED_SESSION; + response.error = ZCONNECTIONLOSS; + + for (auto & callback : path_watches.second) + if (callback) + callback(response); + } + + watches.clear(); + } +} + +void ZooKeeper::AuthRequest::writeImpl(WriteBuffer & out) const +{ + ZooKeeperImpl::write(type, out); + ZooKeeperImpl::write(scheme, out); + ZooKeeperImpl::write(data, out); +} + void ZooKeeper::CreateRequest::writeImpl(WriteBuffer & out) const { ZooKeeperImpl::write(path, out); @@ -974,7 +1035,24 @@ void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in) void ZooKeeper::pushRequest(RequestInfo && info) { - if (!requests.tryPush(info, session_timeout.totalMilliseconds())) + if (expired) + throw Exception("Session expired"); + + info.request->addRootPath(root_path); + info.request->xid = xid.fetch_add(1); + { + std::lock_guard lock(operations_mutex); + operations[info.request->xid] = info; + } + + if (info.watch) + { + info.request->has_watch = true; + std::lock_guard lock(watches_mutex); + watches[info.request->getPath()].emplace_back(std::move(info.watch)); + } + + if (!requests.tryPush(info.request, session_timeout.totalMilliseconds())) throw Exception("Cannot push request to queue within session timeout"); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index a7129351df2..aee18f83110 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -138,6 +138,23 @@ public: using WatchCallback = std::function; + struct AuthRequest final : Request + { + int32_t type = 0; /// ignored by the server + String scheme; + String data; + + OpNum getOpNum() const override { return 100; } + void writeImpl(WriteBuffer &) const override; + ResponsePtr makeResponse() const override; + String getPath() const override { return {}; } + }; + + struct AuthResponse final : Response + { + void readImpl(ReadBuffer &) override {}; + }; + struct CloseRequest final : Request { OpNum getOpNum() const override { return -11; } @@ -169,8 +186,6 @@ public: void removeRootPath(const String & root_path) override; }; - using CreateCallback = std::function; - struct RemoveRequest final : Request { String path; @@ -188,8 +203,6 @@ public: void readImpl(ReadBuffer &) override {} }; - using RemoveCallback = std::function; - struct ExistsRequest final : Request { String path; @@ -208,8 +221,6 @@ public: void readImpl(ReadBuffer &) override; }; - using ExistsCallback = std::function; - struct GetRequest final : Request { String path; @@ -229,8 +240,6 @@ public: void readImpl(ReadBuffer &) override; }; - using GetCallback = std::function; - struct SetRequest final : Request { String path; @@ -251,8 +260,6 @@ public: void readImpl(ReadBuffer &) override; }; - using SetCallback = std::function; - struct ListRequest final : Request { String path; @@ -272,8 +279,6 @@ public: void readImpl(ReadBuffer &) override; }; - using ListCallback = std::function; - struct CheckRequest final : Request { String path; @@ -291,8 +296,6 @@ public: void readImpl(ReadBuffer &) override {}; }; - using CheckCallback = std::function; - struct MultiRequest final : Request { Requests requests; @@ -314,8 +317,6 @@ public: void removeRootPath(const String & root_path) override; }; - using MultiCallback = std::function; - /// Connection to addresses is performed in order. If you want, shuffle them manually. ZooKeeper( @@ -331,6 +332,16 @@ public: /// If not valid, you can only destroy the object. All other methods will throw exception. bool isValid() const { return !expired; } + using CreateCallback = std::function; + using RemoveCallback = std::function; + using ExistsCallback = std::function; + using GetCallback = std::function; + using SetCallback = std::function; + using ListCallback = std::function; + using CheckCallback = std::function; + using MultiCallback = std::function; + + void create( const String & path, const String & data, @@ -377,7 +388,7 @@ public: void close(); - enum ZOO_ERRORS + enum Error { ZOK = 0, @@ -421,6 +432,27 @@ public: static const char * errorMessage(int32_t code); + /// For watches. + enum State + { + EXPIRED_SESSION = -112, + AUTH_FAILED = -113, + CONNECTING = 1, + ASSOCIATING = 2, + CONNECTED = 3, + NOTCONNECTED = 999 + }; + + enum Event + { + CREATED = 1, + DELETED = 2, + CHANGED = 3, + CHILD = 4, + SESSION = -1, + NOTWATCHING = -2 + }; + private: String root_path; ACLs default_acls; @@ -431,6 +463,8 @@ private: std::optional in; std::optional out; + std::atomic xid {1}; /// TODO deal with xid overflow + struct RequestInfo { RequestPtr request; @@ -438,10 +472,10 @@ private: WatchCallback watch; }; - using RequestsQueue = ConcurrentBoundedQueue; + using RequestsQueue = ConcurrentBoundedQueue; RequestsQueue requests{1}; - void pushRequest(RequestInfo && info); + void pushRequest(RequestInfo && request); using Operations = std::map; @@ -467,13 +501,16 @@ private: void sendHandshake(); void receiveHandshake(); - void sendAuth(const String & auth_scheme, const String & auth_data); + void sendAuth(const String & scheme, const String & data); void receiveEvent(); void sendThread(); void receiveThread(); + /// Call all remaining callbacks and watches, passing errors to them. + void finalize(); + template void write(const T &); From c5ddbbb1ab6ef02646fc66cc608af0c9d397864c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 16:31:09 +0300 Subject: [PATCH 25/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index aee18f83110..959a2fecf28 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -12,12 +12,13 @@ #include #include +#include #include #include #include #include -#include #include +#include namespace ZooKeeperImpl From 1b0a43ce55f7724095d197f09dfe06e2fbbad453 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 17:19:32 +0300 Subject: [PATCH 26/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 959a2fecf28..138a9d33a24 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include From 72ededeab5ec769af2fa7b07a7f8e875627e1ddd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 20:45:30 +0300 Subject: [PATCH 27/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 25 +++++-- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- .../ZooKeeper/tests/zkutil_test_commands.cpp | 7 +- .../tests/zkutil_test_commands_new_lib.cpp | 66 +++++++++++++++---- 4 files changed, 78 insertions(+), 22 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 494ea3ce77d..f1560da9be8 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -785,13 +785,24 @@ void ZooKeeper::receiveEvent() auto it = watches.find(watch_response.path); if (it == watches.end()) - throw Exception("Received event for unknown watch"); + { + /// This is Ok. + /// Because watches are identified by path. + /// And there may exist many watches for single path. + /// And watch is added to the list of watches on client side + /// slightly before than it is registered by the server. + /// And that's why new watch may be already fired by old event, + /// but then the server will actually register new watch + /// and will send event again later. + } + else + { + for (auto & callback : it->second) + if (callback) + callback(watch_response); - for (auto & callback : it->second) - if (callback) - callback(watch_response); - - watches.erase(it); + watches.erase(it); + } }; std::cerr << "Received watch\n"; @@ -981,8 +992,8 @@ void ZooKeeper::SetResponse::readImpl(ReadBuffer & in) void ZooKeeper::ListResponse::readImpl(ReadBuffer & in) { - ZooKeeperImpl::read(stat, in); ZooKeeperImpl::read(names, in); + ZooKeeperImpl::read(stat, in); } ZooKeeper::MultiResponse::MultiResponse(const Requests & requests) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 138a9d33a24..777299004fe 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -275,8 +275,8 @@ public: struct ListResponse final : Response { - Stat stat; std::vector names; + Stat stat; void readImpl(ReadBuffer &) override; }; diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp index d7fa1d3bf42..052c920e325 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp @@ -14,7 +14,6 @@ try } ZooKeeper zk(argv[1], "", 5000); - Strings children; std::cout << "create path" << std::endl; zk.create("/test", "old", zkutil::CreateMode::Persistent); @@ -28,6 +27,12 @@ try watch->wait(); std::cout << "watch happened" << std::endl; std::cout << "remove path" << std::endl; + + std::cout << "list path" << std::endl; + Strings children = zk.getChildren("/"); + for (const auto & name : children) + std::cerr << "\t" << name << "\n"; + zk.remove("/test"); Ops ops; diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp index 3bb3f578fa8..f6a9a4fc937 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp @@ -34,45 +34,85 @@ try Strings children; - std::cout << "create path" << '\n'; + std::cout << "create\n"; zk.create("/test", "old", false, false, {}, [](const ZooKeeper::CreateResponse & response) { if (response.error) - std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + std::cerr << "Error (create) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Created path: " << response.path_created << '\n'; }); - std::cout << "get path" << '\n'; + std::cout << "get\n"; zk.get("/test", [](const ZooKeeper::GetResponse & response) { if (response.error) - std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + std::cerr << "Error (get) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Value: " << response.data << '\n'; }, -/* [](const ZooKeeper::WatchResponse & response) + [](const ZooKeeper::WatchResponse & response) { if (response.error) - std::cerr << "Watch, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + std::cerr << "Watch (get) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else - std::cerr << "Watch, path: " << response.path << ", type: " << response.type << '\n'; - }*/ {}); + std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n'; + }); - std::cout << "set path" << '\n'; + std::cout << "set\n"; zk.set("/test", "new", -1, [](const ZooKeeper::SetResponse & response) { if (response.error) - std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + std::cerr << "Error (set) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Set\n"; }); - std::cout << "remove path" << '\n'; + std::cout << "list\n"; + + zk.list("/", + [](const ZooKeeper::ListResponse & response) + { + if (response.error) + std::cerr << "Error (list) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + { + std::cerr << "Children:\n"; + for (const auto & name : response.names) + std::cerr << name << "\n"; + } + }, + [](const ZooKeeper::WatchResponse & response) + { + if (response.error) + std::cerr << "Watch (list) on /, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n'; + }); + + std::cout << "exists\n"; + + zk.exists("/test", + [](const ZooKeeper::ExistsResponse & response) + { + if (response.error) + std::cerr << "Error (exists) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Exists\n"; + }, + [](const ZooKeeper::WatchResponse & response) + { + if (response.error) + std::cerr << "Watch (exists) on /test, Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + else + std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n'; + }); + + std::cout << "remove\n"; zk.remove("/test", -1, [](const ZooKeeper::RemoveResponse & response) { @@ -82,7 +122,7 @@ try std::cerr << "Removed\n"; }); - std::cout << "multi" << '\n'; + std::cout << "multi\n"; ZooKeeper::Requests ops; @@ -109,7 +149,7 @@ try zk.multi(ops, [](const ZooKeeper::MultiResponse & response) { if (response.error) - std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + std::cerr << "Error (multi) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else { for (const auto & elem : response.responses) From 831d38fd59357f7d78ce172e127c6b3f3ac199f1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 22:07:50 +0300 Subject: [PATCH 28/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 144 ++++++++++++------ dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 14 +- .../tests/zkutil_test_commands_new_lib.cpp | 17 ++- 3 files changed, 123 insertions(+), 52 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index f1560da9be8..7ee1932c2ed 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -419,17 +419,20 @@ ZooKeeper::~ZooKeeper() { try { - stop = true; + /// Send close event. This also signals sending thread to wakeup and then stop. + if (!expired) + close(); if (send_thread.joinable()) send_thread.join(); + /// This will also wakeup receiving event. + socket.shutdown(); + if (receive_thread.joinable()) receive_thread.join(); - if (!expired) - close(); - + /// Fire all remaining callbacks and watches. finalize(); } catch (...) @@ -604,22 +607,13 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) } -void ZooKeeper::close() -{ - CloseRequest request; - request.xid = close_xid; - request.write(*out); - expired = true; -} - - void ZooKeeper::sendThread() { auto prev_heartbeat_time = std::chrono::steady_clock::now(); try { - while (!stop) + while (!expired) { auto now = std::chrono::steady_clock::now(); auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(session_timeout.totalMilliseconds() / 3); @@ -631,6 +625,9 @@ void ZooKeeper::sendThread() if (requests.tryPop(request, max_wait)) { request->write(*out); + + if (request->xid == close_xid) + break; } else { @@ -645,11 +642,14 @@ void ZooKeeper::sendThread() catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); - expired = true; - stop = true; } - /// TODO drain queue + expired = true; + + /// Drain queue + RequestPtr request; + while (requests.tryPop(request)) + ; } @@ -657,20 +657,23 @@ void ZooKeeper::receiveThread() { try { - while (!stop) + while (!expired) { if (!in->poll(session_timeout.totalMicroseconds())) throw Exception("Nothing is received in session timeout"); + if (expired) + break; + receiveEvent(); } } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); - expired = true; - stop = true; } + + expired = true; } @@ -696,7 +699,7 @@ ZooKeeper::ResponsePtr ZooKeeper::SetRequest::makeResponse() const { return std: ZooKeeper::ResponsePtr ZooKeeper::ListRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::CheckRequest::makeResponse() const { return std::make_shared(); } ZooKeeper::ResponsePtr ZooKeeper::MultiRequest::makeResponse() const { return std::make_shared(requests); } -ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { throw Exception("Received response for close request"); } +ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { return std::make_shared(); } void addRootPath(String & path, const String & root_path) { @@ -996,6 +999,20 @@ void ZooKeeper::ListResponse::readImpl(ReadBuffer & in) ZooKeeperImpl::read(stat, in); } +void ZooKeeper::ErrorResponse::readImpl(ReadBuffer & in) +{ + int32_t read_error; + ZooKeeperImpl::read(read_error, in); + + if (read_error != error) + throw Exception("Error code in ErrorResponse (" + toString(read_error) + ") doesn't match error code in header (" + toString(error) + ")"); +} + +void ZooKeeper::CloseResponse::readImpl(ReadBuffer &) +{ + throw Exception("Received response for close request"); +} + ZooKeeper::MultiResponse::MultiResponse(const Requests & requests) { responses.reserve(requests.size()); @@ -1006,7 +1023,43 @@ ZooKeeper::MultiResponse::MultiResponse(const Requests & requests) void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in) { - for (const auto & response : responses) + for (auto & response : responses) + { + OpNum op_num; + bool done; + int32_t op_error; + + ZooKeeperImpl::read(op_num, in); + ZooKeeperImpl::read(done, in); + ZooKeeperImpl::read(op_error, in); + + std::cerr << "Received result for multi: " << op_num << "\n"; + + if (done) + throw Exception("Not enough results received for multi transaction"); + + /// op_num == -1 is special for multi transaction. + /// For unknown reason, error code is duplicated in header and in response body. + + if (op_num == -1) + response = std::make_shared(); + + if (op_error) + { + response->error = op_error; + + /// Set error for whole transaction. + /// If some operations fail, ZK send global error as zero and then send details about each operation. + /// It will set error code for first failed operation and it will set special "runtime inconsistency" code for other operations. + if (!error && op_error != ZRUNTIMEINCONSISTENCY) + error = op_error; + } + + if (!op_error || op_num == -1) + response->readImpl(in); + } + + /// Footer. { OpNum op_num; bool done; @@ -1016,31 +1069,13 @@ void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in) ZooKeeperImpl::read(done, in); ZooKeeperImpl::read(error, in); - std::cerr << "Received result for multi: " << op_num << "\n"; - - if (done) - throw Exception("Not enough results received for multi transaction"); - - if (error) - response->error = error; - else - response->readImpl(in); + if (!done) + throw Exception("Too many results received for multi transaction"); + if (op_num != -1) + throw Exception("Unexpected op_num received at the end of results for multi transaction"); + if (error != -1) + throw Exception("Unexpected error value received at the end of results for multi transaction"); } - - OpNum op_num; - bool done; - int32_t error; - - ZooKeeperImpl::read(op_num, in); - ZooKeeperImpl::read(done, in); - ZooKeeperImpl::read(error, in); - - if (!done) - throw Exception("Too many results received for multi transaction"); - if (op_num != -1) - throw Exception("Unexpected op_num received at the end of results for multi transaction"); - if (error != -1) - throw Exception("Unexpected error value received at the end of results for multi transaction"); } @@ -1050,7 +1085,10 @@ void ZooKeeper::pushRequest(RequestInfo && info) throw Exception("Session expired"); info.request->addRootPath(root_path); - info.request->xid = xid.fetch_add(1); + + if (!info.request->xid) + info.request->xid = xid.fetch_add(1); + { std::lock_guard lock(operations_mutex); operations[info.request->xid] = info; @@ -1215,4 +1253,16 @@ void ZooKeeper::multi( } +void ZooKeeper::close() +{ + CloseRequest request; + request.xid = close_xid; + + RequestInfo request_info; + request_info.request = std::make_shared(std::move(request)); + + pushRequest(std::move(request_info)); +} + + } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 777299004fe..262473736b8 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -96,7 +96,7 @@ public: struct Request { - XID xid; + XID xid = 0; bool has_watch = false; virtual ~Request() {}; @@ -165,6 +165,11 @@ public: String getPath() const override { return {}; } }; + struct CloseResponse final : Response + { + void readImpl(ReadBuffer &) override; + }; + struct CreateRequest final : Request { String path; @@ -319,6 +324,12 @@ public: void removeRootPath(const String & root_path) override; }; + /// This response may be received only as an element of responses in MultiResponse. + struct ErrorResponse final : Response + { + void readImpl(ReadBuffer &) override; + }; + /// Connection to addresses is performed in order. If you want, shuffle them manually. ZooKeeper( @@ -493,7 +504,6 @@ private: std::thread send_thread; std::thread receive_thread; - std::atomic stop {false}; std::atomic expired {false}; void connect( diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp index f6a9a4fc937..1af84103f11 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -114,14 +115,22 @@ try std::cout << "remove\n"; - zk.remove("/test", -1, [](const ZooKeeper::RemoveResponse & response) + Poco::Event event(true); + + zk.remove("/test", -1, [&](const ZooKeeper::RemoveResponse & response) { if (response.error) std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Removed\n"; + + event.set(); }); + event.wait(); + + /// Surprising enough, ZooKeeper can execute multi transaction out of order. So, we must to wait for "remove" to execute before sending "multi". + std::cout << "multi\n"; ZooKeeper::Requests ops; @@ -146,7 +155,7 @@ try ops.emplace_back(std::make_shared(std::move(remove_request))); } - zk.multi(ops, [](const ZooKeeper::MultiResponse & response) + zk.multi(ops, [&](const ZooKeeper::MultiResponse & response) { if (response.error) std::cerr << "Error (multi) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; @@ -158,9 +167,11 @@ try std::cerr << "Created path: " << typeid_cast(*response.responses[0]).path_created << '\n'; } + + event.set(); }); - sleep(5); + event.wait(); return 0; } catch (...) From c8f8dc429c23ef85f7d355572f26e5b2bc46fe7c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 22:23:05 +0300 Subject: [PATCH 29/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 5 ++- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 4 +- .../tests/zkutil_test_commands_new_lib.cpp | 44 +++++++++++++------ 3 files changed, 36 insertions(+), 17 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7ee1932c2ed..7a334b93765 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -585,7 +585,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) request.write(*out); int32_t length; - int32_t xid; + XID xid; int64_t zxid; int32_t err; @@ -701,6 +701,7 @@ ZooKeeper::ResponsePtr ZooKeeper::CheckRequest::makeResponse() const { return st ZooKeeper::ResponsePtr ZooKeeper::MultiRequest::makeResponse() const { return std::make_shared(requests); } ZooKeeper::ResponsePtr ZooKeeper::CloseRequest::makeResponse() const { return std::make_shared(); } + void addRootPath(String & path, const String & root_path) { if (path.empty()) @@ -754,7 +755,7 @@ void ZooKeeper::MultiResponse::removeRootPath(const String & root_path) void ZooKeeper::receiveEvent() { int32_t length; - int32_t xid; + XID xid; int64_t zxid; int32_t err; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 262473736b8..2b8358a8b21 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -174,8 +174,8 @@ public: { String path; String data; - bool is_ephemeral; - bool is_sequential; + bool is_ephemeral = false; + bool is_sequential = false; ACLs acls; OpNum getOpNum() const override { return 1; } diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp index 1af84103f11..6dc9e3cb07b 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp @@ -33,27 +33,34 @@ try ZooKeeper zk(addresses, {}, {}, {}, {5, 0}, {0, 50000}); - Strings children; + Poco::Event event(true); std::cout << "create\n"; - zk.create("/test", "old", false, false, {}, [](const ZooKeeper::CreateResponse & response) + zk.create("/test", "old", false, false, {}, + [&](const ZooKeeper::CreateResponse & response) { if (response.error) std::cerr << "Error (create) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Created path: " << response.path_created << '\n'; + + //event.set(); }); + //event.wait(); + std::cout << "get\n"; zk.get("/test", - [](const ZooKeeper::GetResponse & response) + [&](const ZooKeeper::GetResponse & response) { if (response.error) std::cerr << "Error (get) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Value: " << response.data << '\n'; + + //event.set(); }, [](const ZooKeeper::WatchResponse & response) { @@ -63,20 +70,27 @@ try std::cerr << "Watch (get) on /test, path: " << response.path << ", type: " << response.type << '\n'; }); + //event.wait(); + std::cout << "set\n"; - zk.set("/test", "new", -1, [](const ZooKeeper::SetResponse & response) + zk.set("/test", "new", -1, + [&](const ZooKeeper::SetResponse & response) { if (response.error) std::cerr << "Error (set) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Set\n"; + + //event.set(); }); + //event.wait(); + std::cout << "list\n"; zk.list("/", - [](const ZooKeeper::ListResponse & response) + [&](const ZooKeeper::ListResponse & response) { if (response.error) std::cerr << "Error (list) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; @@ -86,6 +100,8 @@ try for (const auto & name : response.names) std::cerr << name << "\n"; } + + //event.set(); }, [](const ZooKeeper::WatchResponse & response) { @@ -95,15 +111,19 @@ try std::cerr << "Watch (list) on /, path: " << response.path << ", type: " << response.type << '\n'; }); + //event.wait(); + std::cout << "exists\n"; zk.exists("/test", - [](const ZooKeeper::ExistsResponse & response) + [&](const ZooKeeper::ExistsResponse & response) { if (response.error) std::cerr << "Error (exists) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Exists\n"; + + //event.set(); }, [](const ZooKeeper::WatchResponse & response) { @@ -113,23 +133,21 @@ try std::cerr << "Watch (exists) on /test, path: " << response.path << ", type: " << response.type << '\n'; }); - std::cout << "remove\n"; + //event.wait(); - Poco::Event event(true); + std::cout << "remove\n"; zk.remove("/test", -1, [&](const ZooKeeper::RemoveResponse & response) { if (response.error) - std::cerr << "Error " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; + std::cerr << "Error (remove) " << response.error << ": " << ZooKeeper::errorMessage(response.error) << '\n'; else std::cerr << "Removed\n"; - event.set(); + //event.set(); }); - event.wait(); - - /// Surprising enough, ZooKeeper can execute multi transaction out of order. So, we must to wait for "remove" to execute before sending "multi". + //event.wait(); std::cout << "multi\n"; From 644c5e67b697348a4a85d89c96742a946f142643 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 19 Mar 2018 22:25:21 +0300 Subject: [PATCH 30/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 ++++ dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 7a334b93765..6e4ea0b808d 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -1088,7 +1088,11 @@ void ZooKeeper::pushRequest(RequestInfo && info) info.request->addRootPath(root_path); if (!info.request->xid) + { info.request->xid = xid.fetch_add(1); + if (info.request->xid < 0) + throw Exception("XID overflow"); + } { std::lock_guard lock(operations_mutex); diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 2b8358a8b21..66ddf3a8a48 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -476,7 +476,7 @@ private: std::optional in; std::optional out; - std::atomic xid {1}; /// TODO deal with xid overflow + std::atomic xid {1}; struct RequestInfo { From 76f9e0032d49d861afc84151a67ef347af2a15c6 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 20 Mar 2018 00:34:56 +0300 Subject: [PATCH 31/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/Lock.cpp | 83 +---- dbms/src/Common/ZooKeeper/Lock.h | 3 - dbms/src/Common/ZooKeeper/Types.h | 185 +--------- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 362 ++++++-------------- dbms/src/Common/ZooKeeper/ZooKeeper.h | 118 +------ dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 3 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 10 +- utils/zookeeper-dump-tree/main.cpp | 4 +- 8 files changed, 137 insertions(+), 631 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/Lock.cpp b/dbms/src/Common/ZooKeeper/Lock.cpp index 62190fef0ef..e6f002c7805 100644 --- a/dbms/src/Common/ZooKeeper/Lock.cpp +++ b/dbms/src/Common/ZooKeeper/Lock.cpp @@ -11,29 +11,14 @@ bool Lock::tryLock() if (tryCheck() != Status::LOCKED_BY_ME) locked.reset(nullptr); } - - if (!locked) + else { - size_t attempt; std::string dummy; - - /// TODO: ошибка. можно создать эфемерную ноду, но при этом не получить подтверждения даже после нескольких попыток. - /// тогда все последующие локи будут неуспешные из-за существования ноды. - int32_t code = zookeeper->tryCreateWithRetries(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy, &attempt); + int32_t code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy); if (code == ZNODEEXISTS) { - if (attempt == 0) - locked.reset(nullptr); - else - { - zkutil::Stat stat; - zookeeper->get(lock_path, &stat); - if (stat.ephemeralOwner == zookeeper->getClientID()) - locked.reset(new ZooKeeperHandler(zookeeper)); - else - locked.reset(nullptr); - } + locked.reset(nullptr); } else if (code == ZOK) { @@ -52,34 +37,8 @@ void Lock::unlock() if (locked) { auto zookeeper = zookeeper_holder->getZooKeeper(); - try - { - if (tryCheck() == Status::LOCKED_BY_ME) - { - size_t attempt; - int32_t code = zookeeper->tryRemoveEphemeralNodeWithRetries(lock_path, -1, &attempt); - - if (attempt) - { - if (code != ZOK) - throw zkutil::KeeperException(code); - } - else - { - if (code == ZNONODE) - LOG_ERROR(log, "Node " << lock_path << " has been already removed. Probably due to network error."); - else if (code != ZOK) - throw zkutil::KeeperException(code); - } - } - } - catch (const zkutil::KeeperException & e) - { - /// если сессия находится в невостанавливаемом состоянии, то эфемерные ноды нам больше не принадлежат - /// и лок через таймаут будет отпущен - if (!e.isUnrecoverable()) - throw; - } + if (tryCheck() == Status::LOCKED_BY_ME) + zookeeper->remove(lock_path, -1); locked.reset(nullptr); } } @@ -97,47 +56,17 @@ Lock::Status Lock::tryCheck() const else { if (stat.ephemeralOwner == zookeeper->getClientID()) - { lock_status = LOCKED_BY_ME; - } else - { lock_status = LOCKED_BY_OTHER; - } } if (locked && lock_status != LOCKED_BY_ME) - LOG_WARNING(log, "Lock is lost. It is normal if session was reinitialized. Path: " << lock_path << "/" << lock_message); + LOG_WARNING(log, "Lock is lost. It is normal if session was expired. Path: " << lock_path << "/" << lock_message); return lock_status; } -std::string Lock::status2String(Status status) -{ - if (status >= END) - throw zkutil::KeeperException("Wrong status code: " + std::to_string(status)); - static const char * names[] = {"Unlocked", "Locked by me", "Locked by other"}; - return names[status]; -} - -void Lock::unlockOrMoveIfFailed(std::vector & failed_to_unlock_locks) -{ - try - { - unlock(); - } - catch (const zkutil::KeeperException & e) - { - if (e.isTemporaryError()) - { - LOG_WARNING(log, "Fail to unlock lock. Move lock to vector to remove later. Path: " << getPath()); - failed_to_unlock_locks.emplace_back(std::move(*this)); - } - else - throw; - } -} - void Lock::unlockAssumeLockNodeRemovedManually() { locked.reset(nullptr); diff --git a/dbms/src/Common/ZooKeeper/Lock.h b/dbms/src/Common/ZooKeeper/Lock.h index 1fca50a5293..17ded48d26b 100644 --- a/dbms/src/Common/ZooKeeper/Lock.h +++ b/dbms/src/Common/ZooKeeper/Lock.h @@ -51,15 +51,12 @@ namespace zkutil UNLOCKED, LOCKED_BY_ME, LOCKED_BY_OTHER, - END }; - std::string status2String(Status status); /// проверяет создана ли эфемерная нода и кто ее владелец. Status tryCheck() const; void unlock(); - void unlockOrMoveIfFailed(std::vector & failed_to_unlock_locks); void unlockAssumeLockNodeRemovedManually(); bool tryLock(); diff --git a/dbms/src/Common/ZooKeeper/Types.h b/dbms/src/Common/ZooKeeper/Types.h index 1938081bb2e..9bd3dd2b557 100644 --- a/dbms/src/Common/ZooKeeper/Types.h +++ b/dbms/src/Common/ZooKeeper/Types.h @@ -3,184 +3,17 @@ #include #include #include -#include +#include #include namespace zkutil { -using ACLPtr = const ACL_vector *; -using Stat = ::Stat; -class ZooKeeper; - - -struct Op -{ -public: - Op() : data(new zoo_op_t) {} - virtual ~Op() {} - - virtual std::shared_ptr clone() const = 0; - - virtual std::string getPath() const = 0; - - virtual std::string describe() const = 0; - - std::unique_ptr data; - - struct Remove; - struct Create; - struct SetData; - struct Check; -}; - -using OpPtr = std::shared_ptr; - - -struct Op::Remove : public Op -{ - Remove(const std::string & path_, int32_t version_) : - path(path_), version(version_) - { - zoo_delete_op_init(data.get(), path.c_str(), version); - } - - OpPtr clone() const override - { - return std::make_shared(path, version); - } - - std::string getPath() const override { return path; } - - std::string describe() const override { return "command: remove, path: " + path; } - -private: - std::string path; - int32_t version; -}; - -struct Op::Create : public Op -{ - Create(const std::string & path_pattern_, const std::string & value_, ACLPtr acl_, int32_t flags_); - - OpPtr clone() const override - { - return std::make_shared(path_pattern, value, acl, flags); - } - - std::string getPathCreated() { return created_path.data(); } - - std::string getPath() const override { return path_pattern; } - - std::string describe() const override - { - return "command: create" - ", path: " + path_pattern + - ", value: " + value; - } - -private: - std::string path_pattern; - std::string value; - ACLPtr acl; - int32_t flags; - std::vector created_path; -}; - -struct Op::SetData : public Op -{ - SetData(const std::string & path_, const std::string & value_, int32_t version_) : - path(path_), value(value_), version(version_) - { - zoo_set_op_init(data.get(), path.c_str(), value.c_str(), value.size(), version, &stat); - } - - OpPtr clone() const override - { - return std::make_shared(path, value, version); - } - - std::string getPath() const override { return path; } - - std::string describe() const override - { - return - "command: set" - ", path: " + path + - ", value: " + value + - ", version: " + std::to_string(data->set_op.version); - } - -private: - std::string path; - std::string value; - int32_t version; - Stat stat; -}; - -struct Op::Check : public Op -{ - Check(const std::string & path_, int32_t version_) : - path(path_), version(version_) - { - zoo_check_op_init(data.get(), path.c_str(), version); - } - - OpPtr clone() const override - { - return std::make_shared(path, version); - } - - std::string getPath() const override { return path; } - - std::string describe() const override { return "command: check, path: " + path; } - -private: - std::string path; - int32_t version; -}; - -using Ops = std::vector; - - -/// C++ version of zoo_op_result_t -struct OpResult -{ - int err; - std::string value; - std::unique_ptr stat; - - /// ZooKeeper is required for correct chroot path prefixes handling - explicit OpResult(const zoo_op_result_t & op_result, const ZooKeeper * zookeeper = nullptr); -}; -using OpResults = std::vector; -using OpResultsPtr = std::shared_ptr; +using Stat = ZooKeeperImpl::ZooKeeper::Stat; using Strings = std::vector; -/// Simple structure to handle transaction execution results -struct MultiTransactionInfo -{ - Ops ops; - int32_t code = ZOK; - OpResultsPtr op_results; - - MultiTransactionInfo() = default; - - MultiTransactionInfo(int32_t code_, const Ops & ops_, const OpResultsPtr & op_results_) - : ops(ops_), code(code_), op_results(op_results_) {} - - bool empty() const - { - return ops.empty(); - } - - /// Returns failed op if zkutil::isUserError(code) is true - const Op & getFailedOp() const; -}; - - namespace CreateMode { extern const int Persistent; @@ -196,18 +29,6 @@ class ZooKeeper; /// Callback to call when the watch fires. /// Because callbacks are called in the single "completion" thread internal to libzookeeper, /// they must execute as quickly as possible (preferably just set some notification). -/// Parameters: -/// zookeeper - zookeeper session to which the fired watch belongs -/// type - event type, one of the *_EVENT constants from zookeeper.h -/// state - session connection state, one of the *_STATE constants from zookeeper.h -/// path - znode path to which the change happened. if event == ZOO_SESSION_EVENT it is either NULL or empty string. -using WatchCallback = std::function; - - -/// Returns first op which code != ZOK or throws an exception -/// ZooKeeper client sets correct OP codes if the transaction fails because of logical (user) errors like ZNODEEXISTS -/// If it is failed because of network error, for example, OP codes is not set. -/// Therefore you should make zkutil::isUserError() check before the function invocation. -size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_return_code); +using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback; } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 746ed4c609a..fcf5e3736ec 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -42,50 +42,22 @@ namespace zkutil { const int CreateMode::Persistent = 0; -const int CreateMode::Ephemeral = ZOO_EPHEMERAL; -const int CreateMode::EphemeralSequential = ZOO_EPHEMERAL | ZOO_SEQUENCE; -const int CreateMode::PersistentSequential = ZOO_SEQUENCE; +const int CreateMode::Ephemeral = 1; +const int CreateMode::PersistentSequential = 2; +const int CreateMode::EphemeralSequential = 3; static void check(int32_t code, const std::string & path) { - if (code != ZOK) + if (code) throw KeeperException(code, path); } -struct WatchContext -{ - /// ZooKeeper instance exists for the entire WatchContext lifetime. - ZooKeeper & zk; - WatchCallback callback; - CurrentMetrics::Increment metric_increment{CurrentMetrics::ZooKeeperWatch}; - - WatchContext(ZooKeeper & zk_, WatchCallback callback_) : zk(zk_), callback(std::move(callback_)) {} - - void process(int32_t event_type, int32_t state, const char * path) - { - if (callback) - callback(zk, event_type, state, path); - } -}; - -void ZooKeeper::processCallback(zhandle_t *, int type, int state, const char * path, void * watcher_ctx) -{ - WatchContext * context = static_cast(watcher_ctx); - context->process(type, state, path); - - /// It is guaranteed that non-ZOO_SESSION_EVENT notification will be delivered only once - /// (https://issues.apache.org/jira/browse/ZOOKEEPER-890) - if (type != ZOO_SESSION_EVENT) - destroyContext(context); -} - void ZooKeeper::init(const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, const std::string & chroot_) { log = &Logger::get("ZooKeeper"); - zoo_set_debug_level(ZOO_LOG_LEVEL_ERROR); hosts = hosts_; identity = identity_; session_timeout_ms = session_timeout_ms_; @@ -95,9 +67,6 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_, impl = zookeeper_init(hosts_for_lib.c_str(), nullptr, session_timeout_ms, nullptr, nullptr, 0); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); - if (!impl) - throw KeeperException("Fail to initialize zookeeper. Hosts are " + hosts_for_lib); - if (!identity.empty()) { auto code = zoo_add_auth(impl, "digest", identity.c_str(), static_cast(identity.size()), nullptr, nullptr); @@ -187,82 +156,37 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std init(args.hosts, args.identity, args.session_timeout_ms, args.chroot); } -WatchCallback ZooKeeper::callbackForEvent(const EventPtr & event) + +static WatchCallback callbackForEvent(const EventPtr & watch) { - WatchCallback callback; - if (event) - { - callback = [e=event](ZooKeeper &, int, int, const char *) mutable - { - if (e) - { - e->set(); - e.reset(); /// The event is set only once, even if the callback can fire multiple times due to session events. - } - }; - } - return callback; + return [watch](const ZooKeeperImpl::ZooKeeper::WatchResponse &) { watch->set(); }; } -WatchContext * ZooKeeper::createContext(WatchCallback && callback) -{ - if (callback) - { - WatchContext * res = new WatchContext(*this, std::move(callback)); - { - std::lock_guard lock(mutex); - watch_context_store.insert(res); - if (watch_context_store.size() % 10000 == 0) - { - LOG_ERROR(log, "There are " << watch_context_store.size() << " active watches. There must be a leak somewhere."); - } - } - return res; - } - else - return nullptr; -} - -void ZooKeeper::destroyContext(WatchContext * context) -{ - if (context) - { - std::lock_guard lock(context->zk.mutex); - context->zk.watch_context_store.erase(context); - } - delete context; -} int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res, - Stat * stat_, + Stat * stat, WatchCallback watch_callback) { - String_vector strings; - int code; - Stat stat; - watcher_fn watcher = watch_callback ? processCallback : nullptr; - WatchContext * context = createContext(std::move(watch_callback)); - code = zoo_wget_children2(impl, path.c_str(), watcher, context, &strings, &stat); - ProfileEvents::increment(ProfileEvents::ZooKeeperGetChildren); - ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); + int32_t code = 0; + Poco::Event event; - if (code == ZOK) + auto callback = [&](const ZooKeeperImpl::ZooKeeper::ListResponse & response) { - if (stat_) - *stat_ = stat; - res.resize(strings.count); - for (int i = 0; i < strings.count; ++i) - res[i] = std::string(strings.data[i]); - deallocate_String_vector(&strings); - } - else - { - /// The call was unsuccessful, so the watch was not set. Destroy the context. - destroyContext(context); - } + code = response.error; + if (!code) + { + res = response.names; + if (stat) + *stat = response.stat; + } + event.set(); + }; + impl.list(path, callback, watch_callback); + event.wait(); return code; } + Strings ZooKeeper::getChildren( const std::string & path, Stat * stat, const EventPtr & watch) { @@ -272,9 +196,9 @@ Strings ZooKeeper::getChildren( } int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res, - Stat * stat_, const EventPtr & watch) + Stat * stat, const EventPtr & watch) { - int32_t code = retry(std::bind(&ZooKeeper::getChildrenImpl, this, std::ref(path), std::ref(res), stat_, callbackForEvent(watch))); + int32_t code = getChildrenImpl(path, res, stat, callbackForEvent(watch)); if (!(code == ZOK || code == ZNONODE)) throw KeeperException(code, path); @@ -284,21 +208,22 @@ int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res, int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created) { - int code; - /// The name of the created node can be longer than path if the sequential node is created. - size_t name_buffer_size = path.size() + SEQUENTIAL_SUFFIX_SIZE; - std::string name_buffer(name_buffer_size, '\0'); + int32_t code = 0; + Poco::Event event; + + auto callback = [&](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) + { + code = response.error; + if (!code) + path_created = response.path_created; + event.set(); + }; + + impl.create(path, data, mode & 1, mode & 2, {}, callback); /// TODO better mode + event.wait(); - code = zoo_create(impl, path.c_str(), data.c_str(), data.size(), getDefaultACL(), mode, name_buffer.data(), name_buffer_size); ProfileEvents::increment(ProfileEvents::ZooKeeperCreate); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code == ZOK) - { - name_buffer.resize(strlen(name_buffer.data())); - path_created = std::move(name_buffer); - } - return code; } @@ -311,7 +236,7 @@ std::string ZooKeeper::create(const std::string & path, const std::string & data int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data, int32_t mode, std::string & path_created) { - int code = createImpl(path, data, mode, path_created); + int32_t code = createImpl(path, data, mode, path_created); if (!(code == ZOK || code == ZNONODE || @@ -328,16 +253,10 @@ int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data, return tryCreate(path, data, mode, path_created); } -int32_t ZooKeeper::tryCreateWithRetries(const std::string & path, const std::string & data, int32_t mode, std::string & path_created, size_t* attempt) -{ - return retry([&path, &data, mode, &path_created, this] { return tryCreate(path, data, mode, path_created); }, attempt); -} - - void ZooKeeper::createIfNotExists(const std::string & path, const std::string & data) { std::string path_created; - int32_t code = retry(std::bind(&ZooKeeper::createImpl, this, std::ref(path), std::ref(data), zkutil::CreateMode::Persistent, std::ref(path_created))); + int32_t code = createImpl(path, data, zkutil::CreateMode::Persistent, path_created); if (code == ZOK || code == ZNODEEXISTS) return; @@ -360,7 +279,19 @@ void ZooKeeper::createAncestors(const std::string & path) int32_t ZooKeeper::removeImpl(const std::string & path, int32_t version) { - int32_t code = zoo_delete(impl, path.c_str(), version); + int32_t code = 0; + Poco::Event event; + + auto callback = [&](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) + { + if (response.error) + code = response.error; + event.set(); + }; + + impl.remove(path, version, callback); + event.wait(); + ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); return code; @@ -371,15 +302,6 @@ void ZooKeeper::remove(const std::string & path, int32_t version) check(tryRemove(path, version), path); } -void ZooKeeper::removeWithRetries(const std::string & path, int32_t version) -{ - size_t attempt; - int code = tryRemoveWithRetries(path, version, &attempt); - - if (!(code == ZOK || (code == ZNONODE && attempt > 0))) - throw KeeperException(code, path); -} - int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version) { int32_t code = removeImpl(path, version); @@ -391,68 +313,35 @@ int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version) return code; } -int32_t ZooKeeper::tryRemoveWithRetries(const std::string & path, int32_t version, size_t * attempt) +int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallback watch_callback) { - int32_t code = retry(std::bind(&ZooKeeper::removeImpl, this, std::ref(path), version), attempt); - if (!(code == ZOK || - code == ZNONODE || - code == ZBADVERSION || - code == ZNOTEMPTY)) + int32_t code = 0; + Poco::Event event; + + auto callback = [&](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) { - throw KeeperException(code, path); - } + code = response.error; + if (!code && stat) + *stat = response.stat; + event.set(); + }; - return code; -} + impl.exists(path, callback, watch_callback); + event.wait(); -int32_t ZooKeeper::tryRemoveEphemeralNodeWithRetries(const std::string & path, int32_t version, size_t * attempt) -{ - try - { - return tryRemoveWithRetries(path, version, attempt); - } - catch (const KeeperException &) - { - /// Set the flag indicating that the session is better treated as expired so that someone - /// recreates it and the ephemeral nodes are indeed deleted. - is_dirty = true; - - throw; - } -} - -int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback) -{ - int32_t code; - Stat stat; - watcher_fn watcher = watch_callback ? processCallback : nullptr; - WatchContext * context = createContext(std::move(watch_callback)); - code = zoo_wexists(impl, path.c_str(), watcher, context, &stat); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code == ZOK) - { - if (stat_) - *stat_ = stat; - } - if (code != ZOK && code != ZNONODE) - { - /// The call was unsuccessful, so the watch was not set. Destroy the context. - destroyContext(context); - } - return code; } -bool ZooKeeper::exists(const std::string & path, Stat * stat_, const EventPtr & watch) +bool ZooKeeper::exists(const std::string & path, Stat * stat, const EventPtr & watch) { - return existsWatch(path, stat_, callbackForEvent(watch)); + return existsWatch(path, stat, callbackForEvent(watch)); } -bool ZooKeeper::existsWatch(const std::string & path, Stat * stat_, const WatchCallback & watch_callback) +bool ZooKeeper::existsWatch(const std::string & path, Stat * stat, const WatchCallback & watch_callback) { - int32_t code = retry(std::bind(&ZooKeeper::existsImpl, this, path, stat_, watch_callback)); + int32_t code = existsImpl(path, stat, watch_callback); if (!(code == ZOK || code == ZNONODE)) throw KeeperException(code, path); @@ -461,43 +350,35 @@ bool ZooKeeper::existsWatch(const std::string & path, Stat * stat_, const WatchC return true; } -int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * stat_, WatchCallback watch_callback) +int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback) { - DB::PODArray buffer; - buffer.resize(MAX_NODE_SIZE); - int buffer_len = MAX_NODE_SIZE; + int32_t code = 0; + Poco::Event event; - int32_t code; - Stat stat; - watcher_fn watcher = watch_callback ? processCallback : nullptr; - WatchContext * context = createContext(std::move(watch_callback)); + auto callback = [&](const ZooKeeperImpl::ZooKeeper::GetResponse & response) + { + code = response.error; + if (!code) + { + res = response.data; + if (stat) + *stat = response.stat; + } + event.set(); + }; + + impl.get(path, callback, watch_callback); + event.wait(); - code = zoo_wget(impl, path.c_str(), watcher, context, buffer.data(), &buffer_len, &stat); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code == ZOK) - { - if (stat_) - *stat_ = stat; - - if (buffer_len < 0) /// This can happen if the node contains NULL. Do not distinguish it from the empty string. - res.clear(); - else - res.assign(buffer.data(), buffer_len); - } - else - { - /// The call was unsuccessful, so the watch was not set. Destroy the context. - destroyContext(context); - } return code; } std::string ZooKeeper::get(const std::string & path, Stat * stat, const EventPtr & watch) { - int code; + int32_t code = 0; std::string res; if (tryGet(path, res, stat, watch, &code)) return res; @@ -505,14 +386,14 @@ std::string ZooKeeper::get(const std::string & path, Stat * stat, const EventPtr throw KeeperException("Can't get data for node " + path + ": node doesn't exist", code); } -bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat_, const EventPtr & watch, int * return_code) +bool ZooKeeper::tryGet(const std::string & path, std::string & res, Stat * stat, const EventPtr & watch, int * return_code) { - return tryGetWatch(path, res, stat_, callbackForEvent(watch), return_code); + return tryGetWatch(path, res, stat, callbackForEvent(watch), return_code); } -bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Stat * stat_, const WatchCallback & watch_callback, int * return_code) +bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Stat * stat, const WatchCallback & watch_callback, int * return_code) { - int32_t code = retry(std::bind(&ZooKeeper::getImpl, this, std::ref(path), std::ref(res), stat_, watch_callback)); + int32_t code = getImpl(path, res, stat, watch_callback); if (!(code == ZOK || code == ZNONODE)) throw KeeperException(code, path); @@ -524,18 +405,24 @@ bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Stat * } int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data, - int32_t version, Stat * stat_) + int32_t version, Stat * stat) { - Stat stat; - int32_t code = zoo_set2(impl, path.c_str(), data.c_str(), data.length(), version, &stat); + int32_t code = 0; + Poco::Event event; + + auto callback = [&](const ZooKeeperImpl::ZooKeeper::SetResponse & response) + { + code = response.error; + if (!code && stat) + *stat = response.stat; + event.set(); + }; + + impl.set(path, data, version, callback); + event.wait(); + ProfileEvents::increment(ProfileEvents::ZooKeeperSet); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code == ZOK) - { - if (stat_) - *stat_ = stat; - } return code; } @@ -546,7 +433,7 @@ void ZooKeeper::set(const std::string & path, const std::string & data, int32_t void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode) { - int code = trySet(path, data, -1); + int32_t code = trySet(path, data, -1); if (code == ZNONODE) { create(path, data, mode); @@ -556,9 +443,9 @@ void ZooKeeper::createOrUpdate(const std::string & path, const std::string & dat } int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, - int32_t version, Stat * stat_) + int32_t version, Stat * stat) { - int32_t code = setImpl(path, data, version, stat_); + int32_t code = setImpl(path, data, version, stat); if (!(code == ZOK || code == ZNONODE || @@ -567,31 +454,12 @@ int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, return code; } -/// Makes deep copy of zoo_op_result_t and removes chroot prefix from paths -static void convertOpResults(const std::vector & op_results_native, OpResultsPtr & out_op_results, - const ZooKeeper * zookeeper = nullptr) -{ - if (!out_op_results) - out_op_results = std::make_shared(); - - out_op_results->reserve(op_results_native.size()); - for (const zoo_op_result_t & res_native : op_results_native) - out_op_results->emplace_back(res_native, zookeeper); -} int32_t ZooKeeper::multiImpl(const Ops & ops_, OpResultsPtr * out_op_results, MultiTransactionInfo * out_info) { if (ops_.empty()) return ZOK; - /// Workaround of the libzookeeper bug. If the session is expired, zoo_multi sometimes - /// segfaults. - /// Possibly, there is a race condition and a segfault is still possible if the session - /// expires between this check and zoo_multi call. - /// TODO: check if the bug is fixed in the latest version of libzookeeper. - if (expired()) - return ZINVALIDSTATE; - size_t count = ops_.size(); std::vector out_results_native(count); @@ -624,7 +492,7 @@ int32_t ZooKeeper::multiImpl(const Ops & ops_, OpResultsPtr * out_op_results, Mu OpResultsPtr ZooKeeper::multi(const Ops & ops) { OpResultsPtr op_results; - int code = multiImpl(ops, &op_results); + int32_t code = multiImpl(ops, &op_results); KeeperMultiException::check(code, ops, op_results); return op_results; } @@ -734,7 +602,7 @@ ZooKeeper::~ZooKeeper() { LOG_INFO(&Logger::get("~ZooKeeper"), "Closing ZooKeeper session"); - int code = zookeeper_close(impl); + int32_t code = zookeeper_close(impl); if (code != ZOK) { LOG_ERROR(&Logger::get("~ZooKeeper"), "Failed to close ZooKeeper session: " << zerror(code)); @@ -766,12 +634,6 @@ ACLPtr ZooKeeper::getDefaultACL() return default_acl; } -void ZooKeeper::setDefaultACL(ACLPtr new_acl) -{ - std::lock_guard lock(mutex); - default_acl = new_acl; -} - std::string ZooKeeper::error2string(int32_t code) { return zerror(code); @@ -1097,7 +959,7 @@ KeeperMultiException::KeeperMultiException(const MultiTransactionInfo & info_, s info_.code), info(info_) {} -void KeeperMultiException::check(int code, const Ops & ops, const OpResultsPtr & op_results) +void KeeperMultiException::check(int32_t code, const Ops & ops, const OpResultsPtr & op_results) { if (code == ZOK) {} else if (zkutil::isUserError(code)) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.h b/dbms/src/Common/ZooKeeper/ZooKeeper.h index 52f1968eba6..0c52727cf33 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.h @@ -29,8 +29,6 @@ namespace zkutil { const UInt32 DEFAULT_SESSION_TIMEOUT = 30000; -const UInt32 MEDIUM_SESSION_TIMEOUT = 120000; -const UInt32 BIG_SESSION_TIMEOUT = 600000; /// Preferred size of multi() command (in number of ops) constexpr size_t MULTI_BATCH_SIZE = 100; @@ -84,20 +82,10 @@ public: /// This object remains unchanged, and the new session is returned. Ptr startNewSession() const; - /// Returns true, if the session has expired forever. - /// This is possible only if the connection has been established, then lost and re-established - /// again, but too late. - /// In contrast, if, for instance, the server name or port is misconfigured, connection - /// attempts will continue indefinitely, expired() will return false and all method calls - /// will raise ConnectionLoss exception. - /// Also returns true if is_dirty flag is set - a request to close the session ASAP. + /// Returns true, if the session has expired. bool expired(); - ACLPtr getDefaultACL(); - - void setDefaultACL(ACLPtr new_acl); - - /// Create a znode. ACL set by setDefaultACL is used (full access to everybody by default). + /// Create a znode. /// Throw an exception if something went wrong. std::string create(const std::string & path, const std::string & data, int32_t mode); @@ -108,12 +96,9 @@ public: /// In case of other errors throws an exception. int32_t tryCreate(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); int32_t tryCreate(const std::string & path, const std::string & data, int32_t mode); - int32_t tryCreateWithRetries(const std::string & path, const std::string & data, int32_t mode, - std::string & path_created, size_t * attempt = nullptr); /// Create a Persistent node. /// Does nothing if the node already exists. - /// Retries on ConnectionLoss or OperationTimeout. void createIfNotExists(const std::string & path, const std::string & data); /// Creates all non-existent ancestors of the given path with empty contents. @@ -123,42 +108,11 @@ public: /// Remove the node if the version matches. (if version == -1, remove any version). void remove(const std::string & path, int32_t version = -1); - /// Removes the node. In case of network errors tries to remove again. - /// ZNONODE error for the second and the following tries is ignored. - void removeWithRetries(const std::string & path, int32_t version = -1); - /// Doesn't throw in the following cases: /// * The node doesn't exist /// * Versions don't match /// * The node has children. int32_t tryRemove(const std::string & path, int32_t version = -1); - /// Retries in case of network errors, returns ZNONODE if the node is already removed. - int32_t tryRemoveWithRetries(const std::string & path, int32_t version = -1, size_t * attempt = nullptr); - - /// The same, but sets is_dirty flag if all removal attempts were unsuccessful. - /// This is needed because the session might still exist after all retries, - /// even if more time than session_timeout has passed. - /// So we do not rely on the ephemeral node being deleted and set is_dirty to - /// try and close the session ASAP. - /** Ridiculously Long Delay to Expire - When disconnects do happen, the common case should be a very* quick - reconnect to another server, but an extended network outage may - introduce a long delay before a client can reconnect to the ZooKeep‐ - er service. Some developers wonder why the ZooKeeper client li‐ - brary doesn’t simply decide at some point (perhaps twice the session - timeout) that enough is enough and kill the session itself. - There are two answers to this. First, ZooKeeper leaves this kind of - policy decision up to the developer. Developers can easily implement - such a policy by closing the handle themselves. Second, when a Zoo‐ - Keeper ensemble goes down, time freezes. Thus, when the ensemble is - brought back up, session timeouts are restarted. If processes using - ZooKeeper hang in there, they may find out that the long timeout was - due to an extended ensemble failure that has recovered and pick right - up where they left off without any additional startup delay. - - ZooKeeper: Distributed Process Coordination p118 - */ - int32_t tryRemoveEphemeralNodeWithRetries(const std::string & path, int32_t version = -1, size_t * attempt = nullptr); bool exists(const std::string & path, Stat * stat = nullptr, const EventPtr & watch = nullptr); bool existsWatch(const std::string & path, Stat * stat, const WatchCallback & watch_callback); @@ -199,8 +153,6 @@ public: /// Throws only if some operation has returned an "unexpected" error /// - an error that would cause the corresponding try- method to throw. int32_t tryMulti(const Ops & ops, OpResultsPtr * out_results = nullptr); - /// Use only with read-only operations. - int32_t tryMultiWithRetries(const Ops & ops, OpResultsPtr * out_results = nullptr, size_t * attempt = nullptr); /// Throws nothing, just alias of multiImpl int32_t tryMultiNoThrow(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr) { @@ -339,34 +291,15 @@ public: using TryRemoveFuture = Future; TryRemoveFuture asyncTryRemove(const std::string & path, int32_t version = -1); - struct OpResultsAndCode - { - OpResultsPtr results; - std::shared_ptr ops_ptr; - int code; - }; using MultiFuture = Future; MultiFuture asyncMulti(const Ops & ops); /// Like the previous one but don't throw any exceptions on future.get() MultiFuture tryAsyncMulti(const Ops & ops); - static std::string error2string(int32_t code); - /// Max size of node contents in bytes. - /// In 3.4.5 max node size is 1Mb. - static const size_t MAX_NODE_SIZE = 1048576; - - /// Length of the suffix that ZooKeeper adds to sequential nodes. - /// In fact it is smaller, but round it up for convenience. - static const size_t SEQUENTIAL_SUFFIX_SIZE = 64; - - - zhandle_t * getHandle() { return impl; } - private: - friend struct WatchContext; friend class EphemeralNodeHolder; friend struct OpResult; @@ -375,34 +308,6 @@ private: void removeChildrenRecursive(const std::string & path); void tryRemoveChildrenRecursive(const std::string & path); - static WatchCallback callbackForEvent(const EventPtr & event); - WatchContext * createContext(WatchCallback && callback); - static void destroyContext(WatchContext * context); - static void processCallback(zhandle_t * zh, int type, int state, const char * path, void * watcher_ctx); - - template - int32_t retry(T && operation, size_t * attempt = nullptr) - { - int32_t code = operation(); - if (attempt) - *attempt = 0; - for (size_t i = 0; (i < retry_num) && (code == ZOPERATIONTIMEOUT || code == ZCONNECTIONLOSS); ++i) - { - if (attempt) - *attempt = i; - - /// If the connection has been lost, wait timeout/3 hoping for connection re-establishment. - static const int MAX_SLEEP_TIME = 10; - if (code == ZCONNECTIONLOSS) - usleep(std::min(session_timeout_ms * 1000u / 3, MAX_SLEEP_TIME * 1000u * 1000u)); - - LOG_WARNING(log, "Error on attempt " << i << ": " << error2string(code) << ". Retry"); - code = operation(); - } - - return code; - } - /// The following methods don't throw exceptions but return error codes. int32_t createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); int32_t removeImpl(const std::string & path, int32_t version = -1); @@ -414,25 +319,16 @@ private: MultiFuture asyncMultiImpl(const zkutil::Ops & ops_, bool throw_exception); + ZooKeeperImpl::ZooKeeper impl; + std::string hosts; std::string identity; int32_t session_timeout_ms; std::string chroot; std::mutex mutex; - ACLPtr default_acl; - zhandle_t * impl; - std::unordered_set watch_context_store; - - /// Retries number in case of OperationTimeout or ConnectionLoss errors. - static constexpr size_t retry_num = 3; Logger * log = nullptr; - - /// If true, there were unsuccessfull attempts to remove ephemeral nodes. - /// It is better to close the session to remove ephemeral nodes with certainty - /// instead of continuing to use re-established session. - bool is_dirty = false; }; @@ -476,11 +372,7 @@ public: { try { - /// Important: if the ZooKeeper is temporarily unavailable, repeated attempts to - /// delete the node are made. - /// Otherwise it is possible that EphemeralNodeHolder is destroyed, - /// but the session has recovered and the node in ZooKeeper remains for the long time. - zookeeper.tryRemoveEphemeralNodeWithRetries(path); + zookeeper.tryRemove(path); } catch (const KeeperException & e) { diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 6e4ea0b808d..48bd0f3a5e6 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -306,7 +306,7 @@ void read(String & s, ReadBuffer & in) int32_t size = 0; read(size, in); if (size < 0) - throw Exception("Negative size"); + throw Exception("Negative size"); /// TODO Actually it means that zookeeper node have NULL value. Maybe better to treat it like empty string. if (size > max_string_size) throw Exception("Too large string size"); /// TODO error code s.resize(size); @@ -555,7 +555,6 @@ void ZooKeeper::receiveHandshake() int32_t handshake_length; int32_t protocol_version_read; int32_t timeout; - int64_t session_id; constexpr int32_t passwd_len = 16; std::array passwd; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 66ddf3a8a48..9fa4677ac15 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -342,8 +342,13 @@ public: ~ZooKeeper(); - /// If not valid, you can only destroy the object. All other methods will throw exception. - bool isValid() const { return !expired; } + + /// If expired, you can only destroy the object. All other methods will throw exception. + bool isExpired() const { return !expired; } + + /// Useful to check owner of ephemeral node. + int64_t getSessionID() const { return session_id; } + using CreateCallback = std::function; using RemoveCallback = std::function; @@ -476,6 +481,7 @@ private: std::optional in; std::optional out; + int64_t session_id = 0; std::atomic xid {1}; struct RequestInfo diff --git a/utils/zookeeper-dump-tree/main.cpp b/utils/zookeeper-dump-tree/main.cpp index 4c051273004..cabfe1c2068 100644 --- a/utils/zookeeper-dump-tree/main.cpp +++ b/utils/zookeeper-dump-tree/main.cpp @@ -2,7 +2,7 @@ #include #include #include -#include +#include /** Outputs paths of all ZK nodes in arbitrary order. Possibly only in specified directory. @@ -106,7 +106,7 @@ try return 1; } - zkutil::ZooKeeper zookeeper_(options.at("address").as()); + ZooKeeperImpl::ZooKeeper zookeeper(options.at("address").as()); zookeeper = &zookeeper_; states.emplace_back(); From d690c241983a0c74a8be8ea5682ab4b44af76510 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Mar 2018 00:40:53 +0300 Subject: [PATCH 32/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/KeeperException.h | 45 +-- dbms/src/Common/ZooKeeper/Types.h | 5 +- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 313 +++++++----------- dbms/src/Common/ZooKeeper/ZooKeeper.h | 121 +------ dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 4 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 4 +- .../gtest_zkutil_test_multi_exception.cpp | 2 +- dbms/src/Interpreters/DDLWorker.cpp | 47 +-- dbms/src/Server/ClusterCopier.cpp | 6 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 13 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 24 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 4 - 12 files changed, 166 insertions(+), 422 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/KeeperException.h b/dbms/src/Common/ZooKeeper/KeeperException.h index 37b0a04b564..70a31949c76 100644 --- a/dbms/src/Common/ZooKeeper/KeeperException.h +++ b/dbms/src/Common/ZooKeeper/KeeperException.h @@ -23,22 +23,13 @@ namespace zkutil /// You should reinitialize ZooKeeper session in case of these errors -inline bool isUnrecoverableErrorCode(int32_t zk_return_code) +inline bool isHardwareError(int32_t zk_return_code) { - return zk_return_code == ZINVALIDSTATE || zk_return_code == ZSESSIONEXPIRED || zk_return_code == ZSESSIONMOVED; -} - -/// Errors related with temporary network problems -inline bool isTemporaryErrorCode(int32_t zk_return_code) -{ - return zk_return_code == ZCONNECTIONLOSS || zk_return_code == ZOPERATIONTIMEOUT; -} - -/// Any error related with network or master election -/// In case of these errors you should retry the query or reinitialize ZooKeeper session (see isUnrecoverable()) -inline bool isHardwareErrorCode(int32_t zk_return_code) -{ - return isUnrecoverableErrorCode(zk_return_code) || isTemporaryErrorCode(zk_return_code); + return zk_return_code == ZINVALIDSTATE + || zk_return_code == ZSESSIONEXPIRED + || zk_return_code == ZSESSIONMOVED + || zk_return_code == ZCONNECTIONLOSS + || zk_return_code == ZOPERATIONTIMEOUT; } /// Valid errors sent from server @@ -73,23 +64,11 @@ public: const char * className() const throw() override { return "zkutil::KeeperException"; } KeeperException * clone() const override { return new KeeperException(*this); } - /// You should reinitialize ZooKeeper session in case of these errors - bool isUnrecoverable() const - { - return isUnrecoverableErrorCode(code); - } - - /// Errors related with temporary network problems - bool isTemporaryError() const - { - return isTemporaryErrorCode(code); - } - /// Any error related with network or master election - /// In case of these errors you should retry the query or reinitialize ZooKeeper session (see isUnrecoverable()) + /// In case of these errors you should reinitialize ZooKeeper session. bool isHardwareError() const { - return isHardwareErrorCode(code); + return zkutil::isHardwareError(code); } const int32_t code; @@ -106,15 +85,15 @@ private: class KeeperMultiException : public KeeperException { public: - MultiTransactionInfo info; + Requests requests; + Responses responses; /// If it is user error throws KeeperMultiException else throws ordinary KeeperException /// If it is ZOK does nothing - static void check(const MultiTransactionInfo & info); - static void check(int code, const Ops & ops, const OpResultsPtr & op_results); + static void check(int32_t code, const Requests & requests, const Responses & responses); protected: - KeeperMultiException(const MultiTransactionInfo & info, size_t failed_op_index); + KeeperMultiException(const Requests & requests, const Responses & responses); }; }; diff --git a/dbms/src/Common/ZooKeeper/Types.h b/dbms/src/Common/ZooKeeper/Types.h index 9bd3dd2b557..bf7cbf09a70 100644 --- a/dbms/src/Common/ZooKeeper/Types.h +++ b/dbms/src/Common/ZooKeeper/Types.h @@ -24,11 +24,12 @@ namespace CreateMode using EventPtr = std::shared_ptr; -class ZooKeeper; - /// Callback to call when the watch fires. /// Because callbacks are called in the single "completion" thread internal to libzookeeper, /// they must execute as quickly as possible (preferably just set some notification). using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback; +using Requests = ZooKeeperImpl::ZooKeeper::Requests; +using Responses = ZooKeeperImpl::ZooKeeper::Responses; + } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index fcf5e3736ec..2a062cf3153 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -256,7 +256,7 @@ int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data, void ZooKeeper::createIfNotExists(const std::string & path, const std::string & data) { std::string path_created; - int32_t code = createImpl(path, data, zkutil::CreateMode::Persistent, path_created); + int32_t code = createImpl(path, data, CreateMode::Persistent, path_created); if (code == ZOK || code == ZNODEEXISTS) return; @@ -439,7 +439,7 @@ void ZooKeeper::createOrUpdate(const std::string & path, const std::string & dat create(path, data, mode); } else if (code != ZOK) - throw zkutil::KeeperException(code, path); + throw KeeperException(code, path); } int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, @@ -455,51 +455,42 @@ int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, } -int32_t ZooKeeper::multiImpl(const Ops & ops_, OpResultsPtr * out_op_results, MultiTransactionInfo * out_info) +int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses) { - if (ops_.empty()) + if (requests.empty()) return ZOK; - size_t count = ops_.size(); - std::vector out_results_native(count); + int32_t code = 0; + Poco::Event event; - /// Copy the struct containing pointers with default copy-constructor. - /// It is safe because it hasn't got a destructor. - std::vector ops; - ops.reserve(ops_.size()); - for (const auto & op : ops_) - ops.push_back(*(op->data)); + auto callback = [&](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) + { + code = response.error; + if (!code) + responses = response.responses; + event.set(); + }; + + impl.multi(requests, callback); + event.wait(); - int32_t code = zoo_multi(impl, static_cast(ops.size()), ops.data(), out_results_native.data()); ProfileEvents::increment(ProfileEvents::ZooKeeperMulti); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (out_op_results || out_info) - { - OpResultsPtr op_results; - convertOpResults(out_results_native, op_results, this); - - if (out_op_results) - *out_op_results = op_results; - - if (out_info) - *out_info = MultiTransactionInfo(code, ops_, op_results); - } - return code; } -OpResultsPtr ZooKeeper::multi(const Ops & ops) +Responses ZooKeeper::multi(const Requests & requests) { - OpResultsPtr op_results; - int32_t code = multiImpl(ops, &op_results); - KeeperMultiException::check(code, ops, op_results); - return op_results; + Responses responses; + int32_t code = multiImpl(requests, responses); + KeeperMultiException::check(code, requests, responses); + return responses; } -int32_t ZooKeeper::tryMulti(const Ops & ops_, OpResultsPtr * out_results_) +int32_t ZooKeeper::tryMulti(const Requests & requests) { - int32_t code = multiImpl(ops_, out_results_); + Responses responses; + int32_t code = multiImpl(requests, responses); if (!(code == ZOK || code == ZNONODE || @@ -511,29 +502,21 @@ int32_t ZooKeeper::tryMulti(const Ops & ops_, OpResultsPtr * out_results_) return code; } -int32_t ZooKeeper::tryMultiWithRetries(const Ops & ops, OpResultsPtr * out_results, size_t * attempt) -{ - int32_t code = retry(std::bind(&ZooKeeper::multiImpl, this, std::ref(ops), out_results, nullptr), attempt); - if (!(code == ZOK || - code == ZNONODE || - code == ZNODEEXISTS || - code == ZNOCHILDRENFOREPHEMERALS || - code == ZBADVERSION || - code == ZNOTEMPTY)) - throw KeeperException(code); - return code; -} void ZooKeeper::removeChildrenRecursive(const std::string & path) { Strings children = getChildren(path); while (!children.empty()) { - zkutil::Ops ops; + Requests ops; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { removeChildrenRecursive(path + "/" + children.back()); - ops.emplace_back(std::make_unique(path + "/" + children.back(), -1)); + + ZooKeeperImpl::ZooKeeper::RemoveRequest request; + request.path = path + "/" + children.back(); + + ops.emplace_back(std::make_shared(std::move(request))); children.pop_back(); } multi(ops); @@ -547,26 +530,26 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path) return; while (!children.empty()) { - zkutil::Ops ops; + Requests ops; Strings batch; for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { batch.push_back(path + "/" + children.back()); children.pop_back(); tryRemoveChildrenRecursive(batch.back()); - ops.emplace_back(std::make_unique(batch.back(), -1)); + + ZooKeeperImpl::ZooKeeper::RemoveRequest request; + request.path = batch.back(); + + ops.emplace_back(std::make_shared(std::move(request))); } /// Try to remove the children with a faster method - in bulk. If this fails, /// this means someone is concurrently removing these children and we will have /// to remove them one by one. if (tryMulti(ops) != ZOK) - { for (const std::string & child : batch) - { tryRemove(child); - } - } } } @@ -587,164 +570,126 @@ void ZooKeeper::waitForDisappear(const std::string & path) { while (true) { - zkutil::EventPtr event = std::make_shared(); + int32_t code = 0; + int32_t event_type = 0; + Poco::Event event; - std::string unused; - /// Use get instead of exists to prevent watch leak if the node has already disappeared. - if (!tryGet(path, unused, nullptr, event)) - break; + auto callback = [&](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) + { + code = response.error; + if (code) + event.set(); + }; - event->wait(); + auto watch = [&](const ZooKeeperImpl::ZooKeeper::WatchResponse & response) + { + code = response.error; + if (!code) + event_type = response.type; + event.set(); + }; + + impl.exists(path, callback, watch); + event.wait(); + + ProfileEvents::increment(ProfileEvents::ZooKeeperExists); + ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); + + if (code == ZNONODE) + return; + + if (code) + throw KeeperException(code, path); + + if (event_type == ZooKeeperImpl::ZooKeeper::DELETED) + return; } } -ZooKeeper::~ZooKeeper() -{ - LOG_INFO(&Logger::get("~ZooKeeper"), "Closing ZooKeeper session"); - - int32_t code = zookeeper_close(impl); - if (code != ZOK) - { - LOG_ERROR(&Logger::get("~ZooKeeper"), "Failed to close ZooKeeper session: " << zerror(code)); - } - - LOG_INFO(&Logger::get("~ZooKeeper"), "Removing " << watch_context_store.size() << " watches"); - - /// Destroy WatchContexts that will never be used. - for (WatchContext * context : watch_context_store) - delete context; - - LOG_INFO(&Logger::get("~ZooKeeper"), "Removed watches"); -} - ZooKeeperPtr ZooKeeper::startNewSession() const { return std::make_shared(hosts, identity, session_timeout_ms, chroot); } -Op::Create::Create(const std::string & path_pattern_, const std::string & value_, ACLPtr acl_, int32_t flags_) - : path_pattern(path_pattern_), value(value_), acl(acl_), flags(flags_), created_path(path_pattern_.size() + ZooKeeper::SEQUENTIAL_SUFFIX_SIZE) -{ - zoo_create_op_init(data.get(), path_pattern.c_str(), value.c_str(), value.size(), acl, flags, created_path.data(), created_path.size()); -} - -ACLPtr ZooKeeper::getDefaultACL() -{ - std::lock_guard lock(mutex); - return default_acl; -} std::string ZooKeeper::error2string(int32_t code) { - return zerror(code); + return ZooKeeperImpl::ZooKeeper::errorMessage(code); } bool ZooKeeper::expired() { - return is_dirty || zoo_state(impl) == ZOO_EXPIRED_SESSION_STATE; + return impl.isExpired(); } Int64 ZooKeeper::getClientID() { - return zoo_client_id(impl)->client_id; + return impl.getSessionID(); } -ZooKeeper::GetFuture ZooKeeper::asyncGet(const std::string & path) +std::future ZooKeeper::asyncGet(const std::string & path) { - GetFuture future { - [path] (int rc, const char * value, int value_len, const Stat * stat) - { - if (rc != ZOK) - throw KeeperException(rc, path); + std::promise promise; + auto future = promise.get_future(); - std::string value_str; - if (value_len > 0) /// May be otherwise of the node contains NULL. We don't distinguish it from the empty string. - value_str = { value, size_t(value_len) }; + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable + { + if (response.error) + promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise.set_value(response); + }; - return ValueAndStat{ value_str, stat ? *stat : Stat() }; - }}; - - int32_t code = zoo_aget( - impl, path.c_str(), 0, - [] (int rc, const char * value, int value_len, const Stat * stat, const void * data) - { - GetFuture::TaskPtr owned_task = std::move(const_cast(*static_cast(data))); - (*owned_task)(rc, value, value_len, stat); - }, - future.task.get()); + impl.get(path, callback, {}); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code != ZOK) - throw KeeperException(code, path); - return future; } -ZooKeeper::TryGetFuture ZooKeeper::asyncTryGet(const std::string & path) + +std::future ZooKeeper::asyncTryGet(const std::string & path) { - TryGetFuture future { - [path] (int rc, const char * value, int value_len, const Stat * stat) - { - if (rc != ZOK && rc != ZNONODE) - throw KeeperException(rc, path); + std::promise promise; + auto future = promise.get_future(); - std::string value_str; - if (value_len > 0) /// May be otherwise of the node contains NULL. We don't distinguish it from the empty string. - value_str = { value, size_t(value_len) }; + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable + { + if (response.error && response.error != ZNONODE) + promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise.set_value(response); + }; - return ValueAndStatAndExists{ value_str, stat ? *stat : Stat(), rc != ZNONODE }; - }}; - - int32_t code = zoo_aget( - impl, path.c_str(), 0, - [] (int rc, const char * value, int value_len, const Stat * stat, const void * data) - { - TryGetFuture::TaskPtr owned_task = std::move(const_cast(*static_cast(data))); - (*owned_task)(rc, value, value_len, stat); - }, - future.task.get()); + impl.get(path, callback, {}); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code != ZOK) - throw KeeperException(code, path); - return future; } -ZooKeeper::ExistsFuture ZooKeeper::asyncExists(const std::string & path) +std::future ZooKeeper::asyncExists(const std::string & path) { - ExistsFuture future { - [path] (int rc, const Stat * stat) - { - if (rc != ZOK && rc != ZNONODE) - throw KeeperException(rc, path); + std::promise promise; + auto future = promise.get_future(); - return StatAndExists{ stat ? *stat : Stat(), rc != ZNONODE }; - }}; + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable + { + if (response.error && response.error != ZNONODE) + promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise.set_value(response); + }; - int32_t code = zoo_aexists( - impl, path.c_str(), 0, - [] (int rc, const Stat * stat, const void * data) - { - ExistsFuture::TaskPtr owned_task = std::move(const_cast(*static_cast(data))); - (*owned_task)(rc, stat); - }, - future.task.get()); + impl.exists(path, callback, {}); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code != ZOK) - throw KeeperException(code, path); - return future; } + ZooKeeper::GetChildrenFuture ZooKeeper::asyncGetChildren(const std::string & path) { GetChildrenFuture future { @@ -838,18 +783,18 @@ ZooKeeper::TryRemoveFuture ZooKeeper::asyncTryRemove(const std::string & path, i return future; } -ZooKeeper::MultiFuture ZooKeeper::asyncMultiImpl(const zkutil::Ops & ops_, bool throw_exception) +ZooKeeper::MultiFuture ZooKeeper::asyncMultiImpl(const Ops & ops_, bool throw_exception) { /// We need to hold all references to ops data until the end of multi callback struct OpsHolder { - std::shared_ptr ops_ptr; + std::shared_ptr ops_ptr; std::shared_ptr> ops_native; std::shared_ptr> op_results_native; } holder; /// Copy ops (swallow copy) - holder.ops_ptr = std::make_shared(ops_); + holder.ops_ptr = std::make_shared(ops_); /// Copy native ops to contiguous vector holder.ops_native = std::make_shared>(); for (const OpPtr & op : *holder.ops_ptr) @@ -863,7 +808,7 @@ ZooKeeper::MultiFuture ZooKeeper::asyncMultiImpl(const zkutil::Ops & ops_, bool convertOpResults(*holder.op_results_native, res.results, zookeeper); res.ops_ptr = holder.ops_ptr; if (throw_exception && rc != ZOK) - throw zkutil::KeeperException(rc); + throw KeeperException(rc); return res; }}; @@ -897,12 +842,12 @@ ZooKeeper::MultiFuture ZooKeeper::asyncMultiImpl(const zkutil::Ops & ops_, bool return future; } -ZooKeeper::MultiFuture ZooKeeper::tryAsyncMulti(const zkutil::Ops & ops) +ZooKeeper::MultiFuture ZooKeeper::tryAsyncMulti(const Ops & ops) { return asyncMultiImpl(ops, false); } -ZooKeeper::MultiFuture ZooKeeper::asyncMulti(const zkutil::Ops & ops) +ZooKeeper::MultiFuture ZooKeeper::asyncMulti(const Ops & ops) { return asyncMultiImpl(ops, true); } @@ -929,30 +874,6 @@ size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_ret } -OpResult::OpResult(const zoo_op_result_t & op_result, const ZooKeeper * zookeeper) - : err(op_result.err) -{ - if (op_result.value) - { - value = std::string(op_result.value, op_result.value + op_result.valuelen); - - /// Current version of libzookeeper does not cut chroot path prefixes - /// We do it here manually - if (zookeeper && !zookeeper->chroot.empty()) - { - if (startsWith(value, zookeeper->chroot)) - value = value.substr(zookeeper->chroot.length()); - else - throw DB::Exception("Expected ZooKeeper path with chroot " + zookeeper->chroot + ", got " + value, - DB::ErrorCodes::LOGICAL_ERROR); - } - } - - if (op_result.stat) - stat = std::make_unique(*op_result.stat); -} - - KeeperMultiException::KeeperMultiException(const MultiTransactionInfo & info_, size_t failed_op_index_) :KeeperException( "Transaction failed at op #" + std::to_string(failed_op_index_) + ": " + info_.ops.at(failed_op_index_)->describe(), @@ -962,22 +883,12 @@ KeeperMultiException::KeeperMultiException(const MultiTransactionInfo & info_, s void KeeperMultiException::check(int32_t code, const Ops & ops, const OpResultsPtr & op_results) { if (code == ZOK) {} - else if (zkutil::isUserError(code)) + else if (isUserError(code)) throw KeeperMultiException(MultiTransactionInfo(code, ops, op_results), getFailedOpIndex(op_results, code)); else throw KeeperException(code); } -void KeeperMultiException::check(const MultiTransactionInfo & info) -{ - if (info.code == ZOK) {} - else if (zkutil::isUserError(info.code)) - throw KeeperMultiException(info, getFailedOpIndex(info.op_results, info.code)); - else - throw KeeperException(info.code); -} - - const Op & MultiTransactionInfo::getFailedOp() const { return *ops.at(getFailedOpIndex(op_results, code)); diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.h b/dbms/src/Common/ZooKeeper/ZooKeeper.h index 0c52727cf33..0081a00ea28 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.h @@ -75,8 +75,6 @@ public: */ ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name); - ~ZooKeeper(); - /// Creates a new session with the same parameters. This method can be used for reconnecting /// after the session has expired. /// This object remains unchanged, and the new session is returned. @@ -149,14 +147,14 @@ public: /// Performs several operations in a transaction. /// Throws on every error. - OpResultsPtr multi(const Ops & ops); + Responses multi(const Requests & requests); /// Throws only if some operation has returned an "unexpected" error /// - an error that would cause the corresponding try- method to throw. - int32_t tryMulti(const Ops & ops, OpResultsPtr * out_results = nullptr); + int32_t tryMulti(const Requests & requests); /// Throws nothing, just alias of multiImpl - int32_t tryMultiNoThrow(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr) + int32_t tryMultiNoThrow(const Requests & requests, Responses & responses) { - return multiImpl(ops, out_op_results, out_info); + return multiImpl(requests, responses); } Int64 getClientID(); @@ -189,113 +187,26 @@ public: /// /// Future should not be destroyed before the result is gotten. - template - class Future - { - friend class ZooKeeper; - private: - using Task = std::packaged_task; - using TaskPtr = std::unique_ptr; - using TaskPtrPtr = std::unique_ptr; + std::future asyncGet(const std::string & path); - /// Everything is complicated. - /// - /// In libzookeeper async interface a function (e.g. zoo_aget) - /// accepts a pointer to a standalone callback function and void* pointer to the context - /// which is then passed to the callback. - /// The caller is responsible for ensuring that the context lives until the callback - /// is finished and we can't simply pass ownership of the context into function object. - /// Instead, we save the context in a Future object and return it to the caller. - /// The context will live until the Future lives. - /// Context data is wrapped in an unique_ptr so that its address (which is passed to - /// libzookeeper) remains unchanged after the Future is returned from the function. - /// - /// The second problem is that after std::promise has been fulfilled, and the user - /// has gotten the result from std::future, the Future object can be destroyed - /// before the std::promise::set_value() call that fulfils the promise completes in another - /// thread. - /// See http://stackoverflow.com/questions/10843304/race-condition-in-pthread-once - /// To combat this we use the second unique_ptr. Inside the callback, the void* context - /// is cast to unique_ptr and moved into the local unique_ptr to prolong the lifetime of - /// the context data. + std::future asyncTryGet(const std::string & path); - TaskPtrPtr task; - std::future future; + std::future asyncExists(const std::string & path); - template - Future(Args &&... args) : - task(std::make_unique(std::make_unique(std::forward(args)...))), - future((*task)->get_future()) {} + std::future asyncGetChildren(const std::string & path); - public: - Result get() - { - return future.get(); - } - - Future(Future &&) = default; - Future & operator= (Future &&) = default; - - ~Future() - { - /// If nobody has waited for the result, we must wait for it before the object is - /// destroyed, because the object contents can still be used in the callback. - if (future.valid()) - future.wait(); - } - }; - - - struct ValueAndStat - { - std::string value; - Stat stat; - }; - - using GetFuture = Future; - GetFuture asyncGet(const std::string & path); - - - struct ValueAndStatAndExists - { - std::string value; - Stat stat; - bool exists; - }; - - using TryGetFuture = Future; - TryGetFuture asyncTryGet(const std::string & path); - - - struct StatAndExists - { - Stat stat; - bool exists; - }; - - using ExistsFuture = Future; - ExistsFuture asyncExists(const std::string & path); - - - using GetChildrenFuture = Future; - GetChildrenFuture asyncGetChildren(const std::string & path); - - - using RemoveFuture = Future; - RemoveFuture asyncRemove(const std::string & path, int32_t version = -1); + std::future asyncRemove(const std::string & path, int32_t version = -1); /// Doesn't throw in the following cases: /// * The node doesn't exist /// * The versions do not match /// * The node has children - using TryRemoveFuture = Future; - TryRemoveFuture asyncTryRemove(const std::string & path, int32_t version = -1); + std::future asyncTryRemove(const std::string & path, int32_t version = -1); + std::future asyncMulti(const Ops & ops); - using MultiFuture = Future; - MultiFuture asyncMulti(const Ops & ops); /// Like the previous one but don't throw any exceptions on future.get() - MultiFuture tryAsyncMulti(const Ops & ops); + std::future tryAsyncMulti(const Ops & ops); static std::string error2string(int32_t code); @@ -310,15 +221,13 @@ private: /// The following methods don't throw exceptions but return error codes. int32_t createImpl(const std::string & path, const std::string & data, int32_t mode, std::string & path_created); - int32_t removeImpl(const std::string & path, int32_t version = -1); + int32_t removeImpl(const std::string & path, int32_t version); int32_t getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback); - int32_t setImpl(const std::string & path, const std::string & data, int32_t version = -1, Stat * stat = nullptr); + int32_t setImpl(const std::string & path, const std::string & data, int32_t version, Stat * stat); int32_t getChildrenImpl(const std::string & path, Strings & res, Stat * stat, WatchCallback watch_callback); - int32_t multiImpl(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr); + int32_t multiImpl(const Requests & requests, Responses & responses); int32_t existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback); - MultiFuture asyncMultiImpl(const zkutil::Ops & ops_, bool throw_exception); - ZooKeeperImpl::ZooKeeper impl; std::string hosts; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 48bd0f3a5e6..16f4ee8608c 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -426,7 +426,7 @@ ZooKeeper::~ZooKeeper() if (send_thread.joinable()) send_thread.join(); - /// This will also wakeup receiving event. + /// This will also wakeup receiving thread. socket.shutdown(); if (receive_thread.joinable()) @@ -1259,6 +1259,8 @@ void ZooKeeper::multi( void ZooKeeper::close() { + /// TODO closed flag or make method private + CloseRequest request; request.xid = close_xid; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 9fa4677ac15..be8fa89c1a5 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -130,8 +130,8 @@ public: struct WatchResponse final : Response { - int32_t type; - int32_t state; + int32_t type = 0; + int32_t state = 0; String path; void readImpl(ReadBuffer &) override; diff --git a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp index 695b9949a66..a4895971b1d 100644 --- a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp +++ b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp @@ -146,7 +146,7 @@ TEST(zkutil, multi_async_libzookeeper_segfault) auto future = zookeeper->asyncMulti(ops); auto res = future.get(); - EXPECT_TRUE(zkutil::isUnrecoverableErrorCode(res.code)); + EXPECT_TRUE(zkutil::isHardwareError(res.code)); } diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 7deabb7872e..d53605e78db 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -817,19 +817,7 @@ String DDLWorker::enqueueQuery(DDLLogEntry & entry) String query_path_prefix = queue_dir + "/query-"; zookeeper->createAncestors(query_path_prefix); - String node_path; - try - { - node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); - } - catch (const zkutil::KeeperException & e) - { - /// TODO: This condition could be relaxed with additional post-checks - if (e.isTemporaryError()) - throw Exception("Unknown status of distributed DDL task", ErrorCodes::UNKNOWN_STATUS_OF_DISTRIBUTED_DDL_TASK); - - throw; - } + String node_path = zookeeper->create(query_path_prefix, entry.toString(), zkutil::CreateMode::PersistentSequential); /// Optional step try @@ -894,29 +882,22 @@ void DDLWorker::run() { if (e.isHardwareError()) { - if (!e.isTemporaryError()) - { - LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false)); + LOG_DEBUG(log, "Recovering ZooKeeper session after: " << getCurrentExceptionMessage(false)); - while (!stop_flag) + while (!stop_flag) + { + try { - try - { - zookeeper = context.getZooKeeper(); - break; - } - catch (...) - { - tryLogCurrentException(__PRETTY_FUNCTION__); - - using namespace std::chrono_literals; - std::this_thread::sleep_for(5s); - } + zookeeper = context.getZooKeeper(); + break; + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + + using namespace std::chrono_literals; + std::this_thread::sleep_for(5s); } - } - else - { - LOG_DEBUG(log, "Retry task processing after: " << getCurrentExceptionMessage(false)); } } else diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index 4233ce70e3f..9437786aca2 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -1676,11 +1676,7 @@ protected: catch (zkutil::KeeperException & e) { future_is_dirty_checker.reset(); - - if (e.isTemporaryError()) - LOG_INFO(log, "ZooKeeper is lagging: " << e.displayText()); - else - throw; + throw; } if (status.exists) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 1d5ec76c652..0c3f72ab338 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -403,18 +403,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } } - else if (zkutil::isTemporaryErrorCode(info.code)) - { - /** If the connection is lost, and we do not know if the changes were applied, you can not delete the local part - * if the changes were applied, the inserted block appeared in `/blocks/`, and it can not be inserted again. - */ - transaction.commit(); - storage.enqueuePartForCheck(part->name, MAX_AGE_OF_LOCAL_PART_THAT_WASNT_ADDED_TO_ZOOKEEPER); - - /// We do not know whether or not data has been inserted. - throw Exception("Unknown status, client must retry. Reason: " + zkutil::ZooKeeper::error2string(info.code), ErrorCodes::UNKNOWN_STATUS_OF_INSERT); - } - else if (zkutil::isUnrecoverableErrorCode(info.code)) + else if (zkutil::isHardwareError(info.code)) { transaction.rollback(); throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index c5d4a41d496..d1f20f69585 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -256,13 +256,6 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z { std::lock_guard lock(pull_logs_to_queue_mutex); - bool dirty_entries_loaded = false; - if (is_dirty) - { - dirty_entries_loaded = load(zookeeper); - is_dirty = false; - } - String index_str = zookeeper->get(replica_path + "/log_pointer"); UInt64 index; @@ -356,20 +349,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z ops.emplace_back(std::make_shared( replica_path + "/min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1)); - try - { - zookeeper->multi(ops); - } - catch (const zkutil::KeeperException & ex) - { - if (ex.isTemporaryError()) - { - LOG_WARNING(log, "Unknown status of queue update, marking queue dirty (will reload on next iteration)."); - is_dirty = true; - } - - throw; - } + zookeeper->multi(ops); /// Now we have successfully updated the queue in ZooKeeper. Update it in RAM. @@ -406,7 +386,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z next_update_event->set(); } - return dirty_entries_loaded || !log_entries.empty(); + return !log_entries.empty(); } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h index 4215a312ee3..59f4efa017a 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.h @@ -52,10 +52,6 @@ private: */ Queue queue; - /// If true, the queue in RAM is possibly out of sync with ZK and we need to reload it. - /// Protected by pull_logs_to_queue_mutex. - bool is_dirty = false; - InsertsByTime inserts_by_time; time_t min_unprocessed_insert_time = 0; time_t max_processed_insert_time = 0; From 6d889da41e56fd8d61675d340c5f153c5c17c8e7 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 22 Mar 2018 15:15:06 +0300 Subject: [PATCH 33/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/KeeperException.h | 27 +- dbms/src/Common/ZooKeeper/Lock.cpp | 4 +- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 318 +++++++----------- dbms/src/Common/ZooKeeper/ZooKeeper.h | 8 +- .../Common/ZooKeeper/ZooKeeperNodeCache.cpp | 12 +- 5 files changed, 153 insertions(+), 216 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/KeeperException.h b/dbms/src/Common/ZooKeeper/KeeperException.h index 70a31949c76..e258e5154c8 100644 --- a/dbms/src/Common/ZooKeeper/KeeperException.h +++ b/dbms/src/Common/ZooKeeper/KeeperException.h @@ -25,21 +25,21 @@ namespace zkutil /// You should reinitialize ZooKeeper session in case of these errors inline bool isHardwareError(int32_t zk_return_code) { - return zk_return_code == ZINVALIDSTATE - || zk_return_code == ZSESSIONEXPIRED - || zk_return_code == ZSESSIONMOVED - || zk_return_code == ZCONNECTIONLOSS - || zk_return_code == ZOPERATIONTIMEOUT; + return zk_return_code == ZooKeeperImpl::ZooKeeper::ZINVALIDSTATE + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONMOVED + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZOPERATIONTIMEOUT; } /// Valid errors sent from server inline bool isUserError(int32_t zk_return_code) { - return zk_return_code == ZNONODE - || zk_return_code == ZBADVERSION - || zk_return_code == ZNOCHILDRENFOREPHEMERALS - || zk_return_code == ZNODEEXISTS - || zk_return_code == ZNOTEMPTY; + return zk_return_code == ZooKeeperImpl::ZooKeeper::ZNONODE + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZBADVERSION + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY; } @@ -51,12 +51,11 @@ private: : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code) { incrementEventCounter(); } public: - explicit KeeperException(const std::string & msg) : KeeperException(msg, ZOK, 0) {} KeeperException(const std::string & msg, const int32_t code) - : KeeperException(msg + " (" + zerror(code) + ")", code, 0) {} - explicit KeeperException(const int32_t code) : KeeperException(zerror(code), code, 0) {} + : KeeperException(msg + " (" + ZooKeeperImpl::ZooKeeper::errorMessage(code) + ")", code, 0) {} + explicit KeeperException(const int32_t code) : KeeperException(ZooKeeperImpl::ZooKeeper::errorMessage(code), code, 0) {} KeeperException(const int32_t code, const std::string & path) - : KeeperException(std::string{zerror(code)} + ", path: " + path, code, 0) {} + : KeeperException(std::string{ZooKeeperImpl::ZooKeeper::errorMessage(code)} + ", path: " + path, code, 0) {} KeeperException(const KeeperException & exc) : DB::Exception(exc), code(exc.code) { incrementEventCounter(); } diff --git a/dbms/src/Common/ZooKeeper/Lock.cpp b/dbms/src/Common/ZooKeeper/Lock.cpp index e6f002c7805..f7e2881f9d3 100644 --- a/dbms/src/Common/ZooKeeper/Lock.cpp +++ b/dbms/src/Common/ZooKeeper/Lock.cpp @@ -16,11 +16,11 @@ bool Lock::tryLock() std::string dummy; int32_t code = zookeeper->tryCreate(lock_path, lock_message, zkutil::CreateMode::Ephemeral, dummy); - if (code == ZNODEEXISTS) + if (code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { locked.reset(nullptr); } - else if (code == ZOK) + else if (code == ZooKeeperImpl::ZooKeeper::ZOK) { locked.reset(new ZooKeeperHandler(zookeeper)); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 2a062cf3153..aa63a98617f 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -3,12 +3,16 @@ #include #include #include +#include + #include #include #include #include #include +#define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000 + namespace ProfileEvents { @@ -63,25 +67,27 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_, session_timeout_ms = session_timeout_ms_; chroot = chroot_; - std::string hosts_for_lib = hosts + chroot; - impl = zookeeper_init(hosts_for_lib.c_str(), nullptr, session_timeout_ms, nullptr, nullptr, 0); + std::vector addresses_strings; + boost::split(addresses_strings, hosts, boost::is_any_of(",")); + ZooKeeperImpl::ZooKeeper::Addresses addresses; + addresses.reserve(addresses_strings.size()); + for (const auto & address_string : addresses_strings) + addresses.emplace_back(address_string); + + impl = std::make_unique( + addresses, + chroot, + identity_.empty() ? "" : "digest", + identity_, + Poco::Timespan(0, session_timeout_ms_ * 1000), + Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000)); + ProfileEvents::increment(ProfileEvents::ZooKeeperInit); - if (!identity.empty()) - { - auto code = zoo_add_auth(impl, "digest", identity.c_str(), static_cast(identity.size()), nullptr, nullptr); - if (code != ZOK) - throw KeeperException("Zookeeper authentication failed. Hosts are " + hosts, code); - - default_acl = &ZOO_CREATOR_ALL_ACL; - } - else - default_acl = &ZOO_OPEN_ACL_UNSAFE; - LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot)); if (!chroot.empty() && !exists("/")) - throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start."); + throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.", ZooKeeperImpl::ZooKeeper::ZNONODE); } ZooKeeper::ZooKeeper(const std::string & hosts, const std::string & identity, @@ -121,7 +127,8 @@ struct ZooKeeperArgs { chroot = config.getString(config_name + "." + key); } - else throw KeeperException(std::string("Unknown key ") + key + " in config file"); + else + throw KeeperException(std::string("Unknown key ") + key + " in config file", ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS); } /// Shuffle the hosts to distribute the load among ZooKeeper nodes. @@ -138,7 +145,7 @@ struct ZooKeeperArgs if (!chroot.empty()) { if (chroot.front() != '/') - throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot); + throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot, ZooKeeperImpl::ZooKeeper::ZBADARGUMENTS); if (chroot.back() == '/') chroot.pop_back(); } @@ -182,7 +189,7 @@ int32_t ZooKeeper::getChildrenImpl(const std::string & path, Strings & res, event.set(); }; - impl.list(path, callback, watch_callback); + impl->list(path, callback, watch_callback); event.wait(); return code; } @@ -200,7 +207,7 @@ int32_t ZooKeeper::tryGetChildren(const std::string & path, Strings & res, { int32_t code = getChildrenImpl(path, res, stat, callbackForEvent(watch)); - if (!(code == ZOK || code == ZNONODE)) + if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE)) throw KeeperException(code, path); return code; @@ -219,7 +226,7 @@ int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data event.set(); }; - impl.create(path, data, mode & 1, mode & 2, {}, callback); /// TODO better mode + impl->create(path, data, mode & 1, mode & 2, {}, callback); /// TODO better mode event.wait(); ProfileEvents::increment(ProfileEvents::ZooKeeperCreate); @@ -238,10 +245,10 @@ int32_t ZooKeeper::tryCreate(const std::string & path, const std::string & data, { int32_t code = createImpl(path, data, mode, path_created); - if (!(code == ZOK || - code == ZNONODE || - code == ZNODEEXISTS || - code == ZNOCHILDRENFOREPHEMERALS)) + if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || + code == ZooKeeperImpl::ZooKeeper::ZNONODE || + code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS || + code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS)) throw KeeperException(code, path); return code; @@ -258,7 +265,7 @@ void ZooKeeper::createIfNotExists(const std::string & path, const std::string & std::string path_created; int32_t code = createImpl(path, data, CreateMode::Persistent, path_created); - if (code == ZOK || code == ZNODEEXISTS) + if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) return; else throw KeeperException(code, path); @@ -289,7 +296,7 @@ int32_t ZooKeeper::removeImpl(const std::string & path, int32_t version) event.set(); }; - impl.remove(path, version, callback); + impl->remove(path, version, callback); event.wait(); ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); @@ -305,10 +312,10 @@ void ZooKeeper::remove(const std::string & path, int32_t version) int32_t ZooKeeper::tryRemove(const std::string & path, int32_t version) { int32_t code = removeImpl(path, version); - if (!(code == ZOK || - code == ZNONODE || - code == ZBADVERSION || - code == ZNOTEMPTY)) + if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || + code == ZooKeeperImpl::ZooKeeper::ZNONODE || + code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || + code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)) throw KeeperException(code, path); return code; } @@ -326,7 +333,7 @@ int32_t ZooKeeper::existsImpl(const std::string & path, Stat * stat, WatchCallba event.set(); }; - impl.exists(path, callback, watch_callback); + impl->exists(path, callback, watch_callback); event.wait(); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); @@ -343,9 +350,9 @@ bool ZooKeeper::existsWatch(const std::string & path, Stat * stat, const WatchCa { int32_t code = existsImpl(path, stat, watch_callback); - if (!(code == ZOK || code == ZNONODE)) + if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE)) throw KeeperException(code, path); - if (code == ZNONODE) + if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) return false; return true; } @@ -367,7 +374,7 @@ int32_t ZooKeeper::getImpl(const std::string & path, std::string & res, Stat * s event.set(); }; - impl.get(path, callback, watch_callback); + impl->get(path, callback, watch_callback); event.wait(); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); @@ -395,13 +402,13 @@ bool ZooKeeper::tryGetWatch(const std::string & path, std::string & res, Stat * { int32_t code = getImpl(path, res, stat, watch_callback); - if (!(code == ZOK || code == ZNONODE)) + if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNONODE)) throw KeeperException(code, path); if (return_code) *return_code = code; - return code == ZOK; + return code == ZooKeeperImpl::ZooKeeper::ZOK; } int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data, @@ -418,7 +425,7 @@ int32_t ZooKeeper::setImpl(const std::string & path, const std::string & data, event.set(); }; - impl.set(path, data, version, callback); + impl->set(path, data, version, callback); event.wait(); ProfileEvents::increment(ProfileEvents::ZooKeeperSet); @@ -434,11 +441,11 @@ void ZooKeeper::set(const std::string & path, const std::string & data, int32_t void ZooKeeper::createOrUpdate(const std::string & path, const std::string & data, int32_t mode) { int32_t code = trySet(path, data, -1); - if (code == ZNONODE) + if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) { create(path, data, mode); } - else if (code != ZOK) + else if (code != ZooKeeperImpl::ZooKeeper::ZOK) throw KeeperException(code, path); } @@ -447,9 +454,9 @@ int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, { int32_t code = setImpl(path, data, version, stat); - if (!(code == ZOK || - code == ZNONODE || - code == ZBADVERSION)) + if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || + code == ZooKeeperImpl::ZooKeeper::ZNONODE || + code == ZooKeeperImpl::ZooKeeper::ZBADVERSION)) throw KeeperException(code, path); return code; } @@ -458,7 +465,7 @@ int32_t ZooKeeper::trySet(const std::string & path, const std::string & data, int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses) { if (requests.empty()) - return ZOK; + return ZooKeeperImpl::ZooKeeper::ZOK; int32_t code = 0; Poco::Event event; @@ -471,7 +478,7 @@ int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses) event.set(); }; - impl.multi(requests, callback); + impl->multi(requests, callback); event.wait(); ProfileEvents::increment(ProfileEvents::ZooKeeperMulti); @@ -492,12 +499,12 @@ int32_t ZooKeeper::tryMulti(const Requests & requests) Responses responses; int32_t code = multiImpl(requests, responses); - if (!(code == ZOK || - code == ZNONODE || - code == ZNODEEXISTS || - code == ZNOCHILDRENFOREPHEMERALS || - code == ZBADVERSION || - code == ZNOTEMPTY)) + if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || + code == ZooKeeperImpl::ZooKeeper::ZNONODE || + code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS || + code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS || + code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || + code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)) throw KeeperException(code); return code; } @@ -526,7 +533,7 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path) void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path) { Strings children; - if (tryGetChildren(path, children) != ZOK) + if (tryGetChildren(path, children) != ZooKeeperImpl::ZooKeeper::ZOK) return; while (!children.empty()) { @@ -547,7 +554,7 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path) /// Try to remove the children with a faster method - in bulk. If this fails, /// this means someone is concurrently removing these children and we will have /// to remove them one by one. - if (tryMulti(ops) != ZOK) + if (tryMulti(ops) != ZooKeeperImpl::ZooKeeper::ZOK) for (const std::string & child : batch) tryRemove(child); } @@ -589,13 +596,13 @@ void ZooKeeper::waitForDisappear(const std::string & path) event.set(); }; - impl.exists(path, callback, watch); + impl->exists(path, callback, watch); event.wait(); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - if (code == ZNONODE) + if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) return; if (code) @@ -619,12 +626,12 @@ std::string ZooKeeper::error2string(int32_t code) bool ZooKeeper::expired() { - return impl.isExpired(); + return impl->isExpired(); } Int64 ZooKeeper::getClientID() { - return impl.getSessionID(); + return impl->getSessionID(); } @@ -641,7 +648,7 @@ std::future ZooKeeper::asyncGet(const std promise.set_value(response); }; - impl.get(path, callback, {}); + impl->get(path, std::move(callback), {}); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); @@ -656,13 +663,13 @@ std::future ZooKeeper::asyncTryGet(const auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable { - if (response.error && response.error != ZNONODE) + if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); else promise.set_value(response); }; - impl.get(path, callback, {}); + impl->get(path, std::move(callback), {}); ProfileEvents::increment(ProfileEvents::ZooKeeperGet); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); @@ -676,13 +683,13 @@ std::future ZooKeeper::asyncExists(con auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable { - if (response.error && response.error != ZNONODE) + if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); else promise.set_value(response); }; - impl.exists(path, callback, {}); + impl->exists(path, std::move(callback), {}); ProfileEvents::increment(ProfileEvents::ZooKeeperExists); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); @@ -690,166 +697,101 @@ std::future ZooKeeper::asyncExists(con } -ZooKeeper::GetChildrenFuture ZooKeeper::asyncGetChildren(const std::string & path) +std::future ZooKeeper::asyncGetChildren(const std::string & path) { - GetChildrenFuture future { - [path] (int rc, const String_vector * strings) - { - if (rc != ZOK) - throw KeeperException(rc, path); + std::promise promise; + auto future = promise.get_future(); - Strings res; - res.resize(strings->count); - for (int i = 0; i < strings->count; ++i) - res[i] = std::string(strings->data[i]); + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable + { + if (response.error) + promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise.set_value(response); + }; - return res; - }}; - - int32_t code = zoo_aget_children( - impl, path.c_str(), 0, - [] (int rc, const String_vector * strings, const void * data) - { - GetChildrenFuture::TaskPtr owned_task = - std::move(const_cast(*static_cast(data))); - (*owned_task)(rc, strings); - }, - future.task.get()); + impl->list(path, std::move(callback), {}); ProfileEvents::increment(ProfileEvents::ZooKeeperGetChildren); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code != ZOK) - throw KeeperException(code, path); - return future; } -ZooKeeper::RemoveFuture ZooKeeper::asyncRemove(const std::string & path, int32_t version) +std::future ZooKeeper::asyncRemove(const std::string & path, int32_t version) { - RemoveFuture future { - [path] (int rc) - { - if (rc != ZOK) - throw KeeperException(rc, path); - }}; + std::promise promise; + auto future = promise.get_future(); - int32_t code = zoo_adelete( - impl, path.c_str(), version, - [] (int rc, const void * data) - { - RemoveFuture::TaskPtr owned_task = - std::move(const_cast(*static_cast(data))); - (*owned_task)(rc); - }, - future.task.get()); + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable + { + if (response.error) + promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise.set_value(response); + }; + + impl->remove(path, version, std::move(callback)); ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code != ZOK) - throw KeeperException(code, path); - return future; } -ZooKeeper::TryRemoveFuture ZooKeeper::asyncTryRemove(const std::string & path, int32_t version) +std::future ZooKeeper::asyncTryRemove(const std::string & path, int32_t version) { - TryRemoveFuture future { - [path] (int rc) - { - if (rc != ZOK && rc != ZNONODE && rc != ZBADVERSION && rc != ZNOTEMPTY) - throw KeeperException(rc, path); + std::promise promise; + auto future = promise.get_future(); - return rc; - }}; + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable + { + if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE && response.error != ZooKeeperImpl::ZooKeeper::ZBADVERSION && response.error != ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) + promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise.set_value(response); + }; - int32_t code = zoo_adelete( - impl, path.c_str(), version, - [] (int rc, const void * data) - { - TryRemoveFuture::TaskPtr owned_task = - std::move(const_cast(*static_cast(data))); - (*owned_task)(rc); - }, - future.task.get()); + impl->remove(path, version, std::move(callback)); ProfileEvents::increment(ProfileEvents::ZooKeeperRemove); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code != ZOK) - throw KeeperException(code, path); - return future; } -ZooKeeper::MultiFuture ZooKeeper::asyncMultiImpl(const Ops & ops_, bool throw_exception) +std::future ZooKeeper::tryAsyncMulti(const Requests & ops) { - /// We need to hold all references to ops data until the end of multi callback - struct OpsHolder + std::promise promise; + auto future = promise.get_future(); + + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable { - std::shared_ptr ops_ptr; - std::shared_ptr> ops_native; - std::shared_ptr> op_results_native; - } holder; + promise.set_value(response); + }; - /// Copy ops (swallow copy) - holder.ops_ptr = std::make_shared(ops_); - /// Copy native ops to contiguous vector - holder.ops_native = std::make_shared>(); - for (const OpPtr & op : *holder.ops_ptr) - holder.ops_native->push_back(*op->data); - /// Allocate native result holders - holder.op_results_native = std::make_shared>(holder.ops_ptr->size()); - - MultiFuture future{ [throw_exception, holder, zookeeper=this] (int rc) { - OpResultsAndCode res; - res.code = rc; - convertOpResults(*holder.op_results_native, res.results, zookeeper); - res.ops_ptr = holder.ops_ptr; - if (throw_exception && rc != ZOK) - throw KeeperException(rc); - return res; - }}; - - if (ops_.empty()) - { - (**future.task)(ZOK); - return future; - } - - /// Workaround of the libzookeeper bug. - /// TODO: check if the bug is fixed in the latest version of libzookeeper. - if (expired()) - throw KeeperException(ZINVALIDSTATE); - - int32_t code = zoo_amulti(impl, static_cast(holder.ops_native->size()), - holder.ops_native->data(), - holder.op_results_native->data(), - [] (int rc, const void * data) - { - MultiFuture::TaskPtr owned_task = - std::move(const_cast(*static_cast(data))); - (*owned_task)(rc); - }, future.task.get()); + impl->multi(ops, std::move(callback)); ProfileEvents::increment(ProfileEvents::ZooKeeperMulti); ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); - - if (code != ZOK) - throw KeeperException(code); - return future; } -ZooKeeper::MultiFuture ZooKeeper::tryAsyncMulti(const Ops & ops) +std::future ZooKeeper::asyncMulti(const Requests & ops) { - return asyncMultiImpl(ops, false); -} + std::promise promise; + auto future = promise.get_future(); -ZooKeeper::MultiFuture ZooKeeper::asyncMulti(const Ops & ops) -{ - return asyncMultiImpl(ops, true); + auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable + { + if (response.error) + promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + else + promise.set_value(response); + }; + + impl->multi(ops, std::move(callback)); + + ProfileEvents::increment(ProfileEvents::ZooKeeperMulti); + ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions); + return future; } @@ -860,7 +802,7 @@ size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_ret for (size_t index = 0; index < op_results->size(); ++index) { - if ((*op_results)[index].err != ZOK) + if ((*op_results)[index].err != ZooKeeperImpl::ZooKeeper::ZOK) return index; } @@ -875,14 +817,12 @@ size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_ret KeeperMultiException::KeeperMultiException(const MultiTransactionInfo & info_, size_t failed_op_index_) - :KeeperException( - "Transaction failed at op #" + std::to_string(failed_op_index_) + ": " + info_.ops.at(failed_op_index_)->describe(), - info_.code), + : KeeperException("Transaction failed at op #" + std::to_string(failed_op_index_) + ": " + info_.ops.at(failed_op_index_)->describe(), info_.code), info(info_) {} -void KeeperMultiException::check(int32_t code, const Ops & ops, const OpResultsPtr & op_results) +void KeeperMultiException::check(int32_t code, const Requests & requests, const Responses & responses) { - if (code == ZOK) {} + if (code == ZooKeeperImpl::ZooKeeper::ZOK) {} else if (isUserError(code)) throw KeeperMultiException(MultiTransactionInfo(code, ops, op_results), getFailedOpIndex(op_results, code)); else diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.h b/dbms/src/Common/ZooKeeper/ZooKeeper.h index 0081a00ea28..fa9c5790970 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.h @@ -203,10 +203,10 @@ public: /// * The node has children std::future asyncTryRemove(const std::string & path, int32_t version = -1); - std::future asyncMulti(const Ops & ops); + std::future asyncMulti(const Requests & ops); /// Like the previous one but don't throw any exceptions on future.get() - std::future tryAsyncMulti(const Ops & ops); + std::future tryAsyncMulti(const Requests & ops); static std::string error2string(int32_t code); @@ -214,7 +214,7 @@ private: friend class EphemeralNodeHolder; friend struct OpResult; - void init(const std::string & hosts, const std::string & identity, int32_t session_timeout_ms, const std::string & chroot); + void init(const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, const std::string & chroot_); void removeChildrenRecursive(const std::string & path); void tryRemoveChildrenRecursive(const std::string & path); @@ -228,7 +228,7 @@ private: int32_t multiImpl(const Requests & requests, Responses & responses); int32_t existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback); - ZooKeeperImpl::ZooKeeper impl; + std::unique_ptr impl; std::string hosts; std::string identity; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp index 9d874b3df5f..e4d2612131c 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperNodeCache.cpp @@ -41,20 +41,18 @@ std::optional ZooKeeperNodeCache::get(const std::string & path) if (nonexistent_nodes.count(path)) return std::nullopt; - auto watch_callback = [context=context](zkutil::ZooKeeper & zookeeper, int type, int state, const char * path) + auto watch_callback = [context=context](const ZooKeeperImpl::ZooKeeper::WatchResponse & response) { - if (!(type != ZOO_SESSION_EVENT || state == ZOO_EXPIRED_SESSION_STATE)) + if (!(response.type != ZooKeeperImpl::ZooKeeper::SESSION || response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION)) return; bool changed = false; { std::lock_guard lock(context->mutex); - if (&zookeeper != context->zookeeper.get()) - return; - if (type != ZOO_SESSION_EVENT) - changed = context->invalidated_paths.emplace(path).second; - else if (state == ZOO_EXPIRED_SESSION_STATE) + if (response.type != ZooKeeperImpl::ZooKeeper::SESSION) + changed = context->invalidated_paths.emplace(response.path).second; + else if (response.state == ZooKeeperImpl::ZooKeeper::EXPIRED_SESSION) { context->zookeeper = nullptr; context->invalidated_paths.clear(); From 6b684fc1a3009f706a1850f2eece49fc219c7249 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Mar 2018 02:15:14 +0300 Subject: [PATCH 34/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/KeeperException.h | 2 +- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 116 ++++++++++---------- dbms/src/Common/ZooKeeper/ZooKeeper.h | 3 - 3 files changed, 57 insertions(+), 64 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/KeeperException.h b/dbms/src/Common/ZooKeeper/KeeperException.h index e258e5154c8..e188ba90d4d 100644 --- a/dbms/src/Common/ZooKeeper/KeeperException.h +++ b/dbms/src/Common/ZooKeeper/KeeperException.h @@ -92,7 +92,7 @@ public: static void check(int32_t code, const Requests & requests, const Responses & responses); protected: - KeeperMultiException(const Requests & requests, const Responses & responses); + KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses); }; }; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index aa63a98617f..e7617b9163e 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -637,15 +637,16 @@ Int64 ZooKeeper::getClientID() std::future ZooKeeper::asyncGet(const std::string & path) { - std::promise promise; - auto future = promise.get_future(); + /// https://stackoverflow.com/questions/25421346/how-to-create-an-stdfunction-from-a-move-capturing-lambda-expression + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable { if (response.error) - promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); else - promise.set_value(response); + promise->set_value(response); }; impl->get(path, std::move(callback), {}); @@ -658,15 +659,15 @@ std::future ZooKeeper::asyncGet(const std std::future ZooKeeper::asyncTryGet(const std::string & path) { - std::promise promise; - auto future = promise.get_future(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable { if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) - promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); else - promise.set_value(response); + promise->set_value(response); }; impl->get(path, std::move(callback), {}); @@ -678,15 +679,15 @@ std::future ZooKeeper::asyncTryGet(const std::future ZooKeeper::asyncExists(const std::string & path) { - std::promise promise; - auto future = promise.get_future(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable { if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) - promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); else - promise.set_value(response); + promise->set_value(response); }; impl->exists(path, std::move(callback), {}); @@ -699,15 +700,15 @@ std::future ZooKeeper::asyncExists(con std::future ZooKeeper::asyncGetChildren(const std::string & path) { - std::promise promise; - auto future = promise.get_future(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable { if (response.error) - promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); else - promise.set_value(response); + promise->set_value(response); }; impl->list(path, std::move(callback), {}); @@ -719,15 +720,15 @@ std::future ZooKeeper::asyncGetChildren( std::future ZooKeeper::asyncRemove(const std::string & path, int32_t version) { - std::promise promise; - auto future = promise.get_future(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable { if (response.error) - promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); else - promise.set_value(response); + promise->set_value(response); }; impl->remove(path, version, std::move(callback)); @@ -739,15 +740,15 @@ std::future ZooKeeper::asyncRemove(con std::future ZooKeeper::asyncTryRemove(const std::string & path, int32_t version) { - std::promise promise; - auto future = promise.get_future(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable { if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE && response.error != ZooKeeperImpl::ZooKeeper::ZBADVERSION && response.error != ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) - promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); else - promise.set_value(response); + promise->set_value(response); }; impl->remove(path, version, std::move(callback)); @@ -759,12 +760,12 @@ std::future ZooKeeper::asyncTryRemove( std::future ZooKeeper::tryAsyncMulti(const Requests & ops) { - std::promise promise; - auto future = promise.get_future(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable { - promise.set_value(response); + promise->set_value(response); }; impl->multi(ops, std::move(callback)); @@ -776,15 +777,15 @@ std::future ZooKeeper::tryAsyncMulti(co std::future ZooKeeper::asyncMulti(const Requests & ops) { - std::promise promise; - auto future = promise.get_future(); + auto promise = std::make_shared>(); + auto future = promise->get_future(); - auto callback = [promise = std::move(promise)](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable + auto callback = [promise](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) mutable { if (response.error) - promise.set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); else - promise.set_value(response); + promise->set_value(response); }; impl->multi(ops, std::move(callback)); @@ -795,43 +796,38 @@ std::future ZooKeeper::asyncMulti(const } -size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_return_code) +size_t getFailedOpIndex(const Responses & responses, int32_t transaction_return_code) { - if (op_results == nullptr || op_results->empty()) - throw DB::Exception("OpResults is empty", DB::ErrorCodes::LOGICAL_ERROR); + if (responses.empty()) + throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR); - for (size_t index = 0; index < op_results->size(); ++index) - { - if ((*op_results)[index].err != ZooKeeperImpl::ZooKeeper::ZOK) + for (size_t index = 0, size = responses.size(); index < size; ++index) + if (responses[index]->error) return index; - } if (!isUserError(transaction_return_code)) - { throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(transaction_return_code) + "' is not valid response code for that", - DB::ErrorCodes::LOGICAL_ERROR); - } + DB::ErrorCodes::LOGICAL_ERROR); throw DB::Exception("There is no failed OpResult", DB::ErrorCodes::LOGICAL_ERROR); } -KeeperMultiException::KeeperMultiException(const MultiTransactionInfo & info_, size_t failed_op_index_) - : KeeperException("Transaction failed at op #" + std::to_string(failed_op_index_) + ": " + info_.ops.at(failed_op_index_)->describe(), info_.code), - info(info_) {} +KeeperMultiException::KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses) + : KeeperException("Transaction failed at op #" + std::to_string(getFailedOpIndex(responses, code)), code), + requests(requests), responses(responses) +{ +} void KeeperMultiException::check(int32_t code, const Requests & requests, const Responses & responses) { - if (code == ZooKeeperImpl::ZooKeeper::ZOK) {} - else if (isUserError(code)) - throw KeeperMultiException(MultiTransactionInfo(code, ops, op_results), getFailedOpIndex(op_results, code)); + if (!code) + return; + + if (isUserError(code)) + throw KeeperMultiException(code, requests, responses); else throw KeeperException(code); } -const Op & MultiTransactionInfo::getFailedOp() const -{ - return *ops.at(getFailedOpIndex(op_results, code)); -} - } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.h b/dbms/src/Common/ZooKeeper/ZooKeeper.h index fa9c5790970..5c353e12661 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.h @@ -33,8 +33,6 @@ const UInt32 DEFAULT_SESSION_TIMEOUT = 30000; /// Preferred size of multi() command (in number of ops) constexpr size_t MULTI_BATCH_SIZE = 100; -struct WatchContext; -struct MultiTransactionInfo; /// ZooKeeper session. The interface is substantially different from the usual libzookeeper API. /// @@ -212,7 +210,6 @@ public: private: friend class EphemeralNodeHolder; - friend struct OpResult; void init(const std::string & hosts_, const std::string & identity_, int32_t session_timeout_ms_, const std::string & chroot_); From 10be6ca8f8e601b9d3f2b143213a93c5652b198b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Mar 2018 03:45:04 +0300 Subject: [PATCH 35/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/Increment.h | 4 +- dbms/src/Common/ZooKeeper/LeaderElection.h | 32 --- dbms/src/Common/ZooKeeper/Types.h | 24 +++ dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 35 ++++ .../gtest_zkutil_test_multi_exception.cpp | 51 +++-- .../tests/zkutil_expiration_test.cpp | 8 +- .../ZooKeeper/tests/zkutil_test_commands.cpp | 12 +- dbms/src/Interpreters/DDLWorker.cpp | 41 ++-- dbms/src/Server/ClusterCopier.cpp | 51 +++-- .../MergeTree/AbandonableLockInZooKeeper.h | 21 +- .../ReplicatedMergeTreeAlterThread.cpp | 8 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 42 ++-- .../ReplicatedMergeTreeCleanupThread.cpp | 8 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 26 +-- .../ReplicatedMergeTreeRestartingThread.cpp | 17 +- dbms/src/Storages/StorageDistributed.cpp | 11 +- .../Storages/StorageReplicatedMergeTree.cpp | 190 +++++++++--------- .../src/Storages/StorageReplicatedMergeTree.h | 6 +- 18 files changed, 305 insertions(+), 282 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/Increment.h b/dbms/src/Common/ZooKeeper/Increment.h index c4f25be1c70..afcbd7fe205 100644 --- a/dbms/src/Common/ZooKeeper/Increment.h +++ b/dbms/src/Common/ZooKeeper/Increment.h @@ -29,11 +29,11 @@ public: if (zookeeper->tryGet(path, result_str, &stat)) { result = std::stol(result_str) + 1; - success = zookeeper->trySet(path, std::to_string(result), stat.version) == ZOK; + success = zookeeper->trySet(path, std::to_string(result), stat.version) == ZooKeeperImpl::ZooKeeper::ZOK; } else { - success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZOK; + success = zookeeper->tryCreate(path, std::to_string(result), zkutil::CreateMode::Persistent) == ZooKeeperImpl::ZooKeeper::ZOK; } } while (!success); diff --git a/dbms/src/Common/ZooKeeper/LeaderElection.h b/dbms/src/Common/ZooKeeper/LeaderElection.h index 60fc5b4023f..5c91eb15e82 100644 --- a/dbms/src/Common/ZooKeeper/LeaderElection.h +++ b/dbms/src/Common/ZooKeeper/LeaderElection.h @@ -76,41 +76,9 @@ private: std::string node_path = node->getPath(); node_name = node_path.substr(node_path.find_last_of('/') + 1); - cleanOldEphemeralNodes(); - thread = std::thread(&LeaderElection::threadFunction, this); } - void cleanOldEphemeralNodes() - { - if (identifier.empty()) - return; - - /** If there are nodes with same identifier, remove them. - * Such nodes could still be alive after failed attempt of removal, - * if it was temporary communication failure, that was continued for more than session timeout, - * but ZK session is still alive for unknown reason, and someone still holds that ZK session. - * See comments in destructor of EphemeralNodeHolder. - */ - Strings brothers = zookeeper.getChildren(path); - for (const auto & brother : brothers) - { - if (brother == node_name) - continue; - - std::string brother_path = path + "/" + brother; - std::string brother_identifier = zookeeper.get(brother_path); - - if (brother_identifier == identifier) - { - ProfileEvents::increment(ProfileEvents::ObsoleteEphemeralNode); - LOG_WARNING(&Logger::get("LeaderElection"), "Found obsolete ephemeral node for identifier " - + identifier + ", removing: " + brother_path); - zookeeper.tryRemoveWithRetries(brother_path); - } - } - } - void releaseNode() { shutdown = true; diff --git a/dbms/src/Common/ZooKeeper/Types.h b/dbms/src/Common/ZooKeeper/Types.h index bf7cbf09a70..05419f623ef 100644 --- a/dbms/src/Common/ZooKeeper/Types.h +++ b/dbms/src/Common/ZooKeeper/Types.h @@ -29,7 +29,31 @@ using EventPtr = std::shared_ptr; /// they must execute as quickly as possible (preferably just set some notification). using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback; +using RequestPtr = ZooKeeperImpl::ZooKeeper::RequestPtr; +using ResponsePtr = ZooKeeperImpl::ZooKeeper::ResponsePtr; + using Requests = ZooKeeperImpl::ZooKeeper::Requests; using Responses = ZooKeeperImpl::ZooKeeper::Responses; +using CreateRequest = ZooKeeperImpl::ZooKeeper::CreateRequest; +using RemoveRequest = ZooKeeperImpl::ZooKeeper::RemoveRequest; +using ExistsRequest = ZooKeeperImpl::ZooKeeper::ExistsRequest; +using GetRequest = ZooKeeperImpl::ZooKeeper::GetRequest; +using SetRequest = ZooKeeperImpl::ZooKeeper::SetRequest; +using ListRequest = ZooKeeperImpl::ZooKeeper::ListRequest; +using CheckRequest = ZooKeeperImpl::ZooKeeper::CheckRequest; + +using CreateResponse = ZooKeeperImpl::ZooKeeper::CreateResponse; +using RemoveResponse = ZooKeeperImpl::ZooKeeper::RemoveResponse; +using ExistsResponse = ZooKeeperImpl::ZooKeeper::ExistsResponse; +using GetResponse = ZooKeeperImpl::ZooKeeper::GetResponse; +using SetResponse = ZooKeeperImpl::ZooKeeper::SetResponse; +using ListResponse = ZooKeeperImpl::ZooKeeper::ListResponse; +using CheckResponse = ZooKeeperImpl::ZooKeeper::CheckResponse; + +RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode); +RequestPtr makeRemoveRequest(const std::string & path, int version); +RequestPtr makeSetRequest(const std::string & path, const std::string & data); +RequestPtr makeCheckRequest(const std::string & path, int version); + } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index e7617b9163e..e4bf8b2f936 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -830,4 +830,39 @@ void KeeperMultiException::check(int32_t code, const Requests & requests, const throw KeeperException(code); } + +RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode) +{ + auto request = std::make_shared(); + request->path = path; + request->data = data; + request->is_ephemeral = create_mode == CreateMode::Ephemeral || create_mode == CreateMode::EphemeralSequential; + request->is_sequential = create_mode == CreateMode::PersistentSequential || create_mode == CreateMode::EphemeralSequential; + return request; +} + +RequestPtr makeRemoveRequest(const std::string & path, int version) +{ + auto request = std::make_shared(); + request->path = path; + request->version = version; + return request; +} + +RequestPtr makeSetRequest(const std::string & path, const std::string & data) +{ + auto request = std::make_shared(); + request->path = path; + request->data = data; + return request; +} + +RequestPtr makeCheckRequest(const std::string & path, int version) +{ + auto request = std::make_shared(); + request->path = path; + request->version = version; + return request; +} + } diff --git a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp index a4895971b1d..2119f52e2c1 100644 --- a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp +++ b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp @@ -31,25 +31,24 @@ TEST(zkutil, multi_nice_exception_msg) { auto zookeeper = std::make_unique("localhost:2181"); - auto acl = zookeeper->getDefaultACL(); - zkutil::Ops ops; + zkutil::Requests ops; ASSERT_NO_THROW( zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi"); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); zookeeper->multi(ops); ); try { ops.clear(); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/c", "_", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi/c", -1)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "BadBoy", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/b", "_", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/c", "_", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi/c", -1)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "BadBoy", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/b", "_", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); zookeeper->multi(ops); FAIL(); @@ -69,8 +68,7 @@ TEST(zkutil, multi_nice_exception_msg) TEST(zkutil, multi_async) { auto zookeeper = std::make_unique("localhost:2181"); - auto acl = zookeeper->getDefaultACL(); - zkutil::Ops ops; + zkutil::Requests ops; zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi"); @@ -81,14 +79,14 @@ TEST(zkutil, multi_async) { ops.clear(); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "", acl, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "", zkutil::CreateMode::Persistent)); auto fut = zookeeper->tryAsyncMulti(ops); ops.clear(); auto res = fut.get(); - ASSERT_TRUE(res.code == ZOK); + ASSERT_TRUE(res.code == ZooKeeperImpl::ZooKeeper::ZOK); ASSERT_EQ(res.results->size(), 2); ASSERT_EQ(res.ops_ptr->size(), 2); } @@ -100,11 +98,11 @@ TEST(zkutil, multi_async) for (size_t i = 0; i < 10000; ++i) { ops.clear(); - ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi", -1)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", -1)); - ops.emplace_back(new zkutil::Op::SetData("/clickhouse_test/zkutil_multi", "xxx", 42)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest("/clickhouse_test/zkutil_multi", -1)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", -1)); + ops.emplace_back(zkutil::makeSetRequest("/clickhouse_test/zkutil_multi", "xxx", 42)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); futures.emplace_back(zookeeper->asyncMulti(ops)); } @@ -118,14 +116,14 @@ TEST(zkutil, multi_async) { ops.clear(); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi", "_", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest("/clickhouse_test/zkutil_multi/a", "_", zkutil::CreateMode::Persistent)); auto fut = zookeeper->tryAsyncMulti(ops); ops.clear(); auto res = fut.get(); - ASSERT_TRUE(res.code == ZNODEEXISTS); + ASSERT_TRUE(res.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS); ASSERT_EQ(res.results->size(), 2); ASSERT_EQ(res.ops_ptr->size(), 2); } @@ -135,9 +133,9 @@ TEST(zkutil, multi_async) TEST(zkutil, multi_async_libzookeeper_segfault) { auto zookeeper = std::make_unique("localhost:2181", "", 1000); - zkutil::Ops ops; + zkutil::Requests ops; - ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", 0)); + ops.emplace_back(zkutil::makeCheckRequest("/clickhouse_test/zkutil_multi", 0)); /// Uncomment to test //auto cmd = ShellCommand::execute("sudo service zookeeper restart"); @@ -159,15 +157,14 @@ TEST(zkutil, multi_create_sequential) zookeeper->createAncestors("/clickhouse_test/"); zookeeper = std::make_unique("localhost:2181", "", zkutil::DEFAULT_SESSION_TIMEOUT, "/clickhouse_test"); - auto acl = zookeeper->getDefaultACL(); - zkutil::Ops ops; + zkutil::Requests ops; String base_path = "/zkutil/multi_create_sequential"; zookeeper->tryRemoveRecursive(base_path); zookeeper->createAncestors(base_path + "/"); String sequential_node_prefix = base_path + "/queue-"; - ops.emplace_back(new zkutil::Op::Create(sequential_node_prefix, "", acl, zkutil::CreateMode::EphemeralSequential)); + ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential)); zkutil::OpResultsPtr results = zookeeper->multi(ops); zkutil::OpResult & sequential_node_result_op = results->at(0); diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp index 073e482e109..2c98847213c 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp @@ -33,9 +33,9 @@ int main(int argc, char ** argv) while (true) { { - zkutil::Ops ops; - ops.emplace_back(std::make_shared("/test/zk_expiration_test", "hello", zk.getDefaultACL(), zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared("/test/zk_expiration_test", -1)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest>("/test/zk_expiration_test", "hello", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest>("/test/zk_expiration_test", -1)); zkutil::MultiTransactionInfo info; zk.tryMultiNoThrow(ops, nullptr, &info); @@ -43,7 +43,7 @@ int main(int argc, char ** argv) std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(info.code) << std::endl; try { - if (info.code != ZOK) + if (info.code) std::cout << "Path: " << info.getFailedOp().getPath() << std::endl; } catch (...) diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp index 052c920e325..30d654efdd7 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp @@ -35,13 +35,13 @@ try zk.remove("/test"); - Ops ops; - ops.emplace_back(std::make_unique("/test", "multi1", zk.getDefaultACL(), CreateMode::Persistent)); - ops.emplace_back(std::make_unique("/test", "multi2", -1)); - ops.emplace_back(std::make_unique("/test", -1)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest("/test", "multi1", CreateMode::Persistent)); + ops.emplace_back(zkutil::makeSetRequest("/test", "multi2", -1)); + ops.emplace_back(zkutil::makeRemoveRequest("/test", -1)); std::cout << "multi" << std::endl; - OpResultsPtr res = zk.multi(ops); - std::cout << "path created: " << dynamic_cast(*ops[0]).getPathCreated() << std::endl; + zkutil::Responses res = zk.multi(ops); + std::cout << "path created: " << dynamic_cast(*ops[0]).path_created << std::endl; return 0; } diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index d53605e78db..36eeae90272 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -549,16 +549,16 @@ void DDLWorker::processTask(DDLTask & task) String active_node_path = task.entry_path + "/active/" + task.host_id_str; String finished_node_path = task.entry_path + "/finished/" + task.host_id_str; - auto code = zookeeper->tryCreateWithRetries(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy); - if (code == ZOK || code == ZNODEEXISTS) + auto code = zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy); + if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { // Ok } - else if (code == ZNONODE) + else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) { /// There is no parent createStatusDirs(task.entry_path); - if (ZOK != zookeeper->tryCreateWithRetries(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy)) + if (ZooKeeperImpl::ZooKeeper::ZOK != zookeeper->tryCreate(active_node_path, "", zkutil::CreateMode::Ephemeral, dummy)) throw zkutil::KeeperException(code, active_node_path); } else @@ -599,10 +599,9 @@ void DDLWorker::processTask(DDLTask & task) /// FIXME: if server fails right here, the task will be executed twice. We need WAL here. /// Delete active flag and create finish flag - zkutil::Ops ops; - ops.emplace_back(std::make_shared(active_node_path, -1)); - ops.emplace_back(std::make_shared(finished_node_path, task.execution_status.serializeText(), - zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeRemoveRequest(active_node_path, -1)); + ops.emplace_back(zkutil::makeCreateRequest(finished_node_path, task.execution_status.serializeText(), zkutil::CreateMode::Persistent)); zookeeper->multi(ops); } @@ -779,9 +778,9 @@ void DDLWorker::cleanupQueue() } /// Remove the lock node and its parent atomically - zkutil::Ops ops; - ops.emplace_back(std::make_shared(lock_path, -1)); - ops.emplace_back(std::make_shared(node_path, -1)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeRemoveRequest(lock_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(node_path, -1)); zookeeper->multi(ops); lock->unlockAssumeLockNodeRemovedManually(); @@ -798,13 +797,19 @@ void DDLWorker::cleanupQueue() /// Try to create nonexisting "status" dirs for a node void DDLWorker::createStatusDirs(const std::string & node_path) { - zkutil::Ops ops; - auto acl = zookeeper->getDefaultACL(); - ops.emplace_back(std::make_shared(node_path + "/active", "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(node_path + "/finished", "", acl, zkutil::CreateMode::Persistent)); - + zkutil::Requests ops; + { + zkutil::CreateRequest request; + request.path = node_path + "/active"; + ops.emplace_back(std::make_shared(std::move(request))); + } + { + zkutil::CreateRequest request; + request.path = node_path + "/finished"; + ops.emplace_back(std::make_shared(std::move(request))); + } int code = zookeeper->tryMulti(ops); - if (code != ZOK && code != ZNODEEXISTS) + if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw zkutil::KeeperException(code); } @@ -1041,7 +1046,7 @@ private: { Strings res; int code = zookeeper->tryGetChildren(node_path, res); - if (code != ZOK && code != ZNONODE) + if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE) throw zkutil::KeeperException(code, node_path); return res; } diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index 9437786aca2..d46b91165f2 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -359,18 +359,26 @@ struct TaskCluster static zkutil::MultiTransactionInfo checkNoNodeAndCommit( const zkutil::ZooKeeperPtr & zookeeper, const String & checking_node_path, - zkutil::OpPtr && op) + zkutil::RequestPtr && op) { - zkutil::Ops ops; - ops.emplace_back(std::make_shared(checking_node_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(checking_node_path, -1)); + zkutil::Requests ops; + { + zkutil::CreateRequest request; + request.path = checking_node_path; + ops.emplace_back(std::make_shared(std::move(request))); + } + { + zkutil::RemoveRequest request; + request.path = checking_node_path; + ops.emplace_back(std::make_shared(std::move(request))); + } ops.emplace_back(std::move(op)); - zkutil::MultiTransactionInfo info; - zookeeper->tryMultiNoThrow(ops, nullptr, &info); + zkutil::Responses responses; + auto code = zookeeper->tryMultiNoThrow(ops, responses); - if (info.code != ZOK && !zkutil::isUserError(info.code)) - throw zkutil::KeeperException(info.code); + if (code && !zkutil::isUserError(code)) + throw zkutil::KeeperException(code); return info; } @@ -896,7 +904,7 @@ public: int code; zookeeper->tryGetWatch(task_description_path, task_config_str, &stat, task_description_watch_callback, &code); - if (code != ZOK) + if (code) throw Exception("Can't get description node " + task_description_path, ErrorCodes::BAD_ARGUMENTS); LOG_DEBUG(log, "Loading description, zxid=" << task_descprtion_current_stat.czxid); @@ -1050,15 +1058,15 @@ protected: } else { - zkutil::Ops ops; - ops.emplace_back(new zkutil::Op::SetData(workers_version_path, description, version)); - ops.emplace_back(new zkutil::Op::Create(current_worker_path, description, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); + ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); auto code = zookeeper->tryMulti(ops); - if (code == ZOK || code == ZNODEEXISTS) + if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) return std::make_shared(current_worker_path, *zookeeper, false, false, description); - if (code == ZBADVERSION) + if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) { LOG_DEBUG(log, "A concurrent worker has just been added, will check free worker slots again"); } @@ -1212,7 +1220,7 @@ protected: } catch (zkutil::KeeperException & e) { - if (e.code == ZNODEEXISTS) + if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { LOG_DEBUG(log, "Partition " << task_partition.name << " is cleaning now by somebody, sleep"); std::this_thread::sleep_for(default_sleep_time); @@ -1447,7 +1455,6 @@ protected: ClusterPartition & cluster_partition = task_table.getClusterPartition(task_partition.name); auto zookeeper = getZooKeeper(); - auto acl = zookeeper->getDefaultACL(); String is_dirty_flag_path = task_partition.getCommonPartitionIsDirtyPath(); String current_task_is_active_path = task_partition.getActiveWorkerPath(); @@ -1459,7 +1466,7 @@ protected: auto create_is_dirty_node = [&] () { auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent); - if (code != ZOK && code != ZNODEEXISTS) + if (code && code != ZNODEEXISTS) throw zkutil::KeeperException(code, is_dirty_flag_path); }; @@ -1510,7 +1517,7 @@ protected: } catch (const zkutil::KeeperException & e) { - if (e.code == ZNODEEXISTS) + if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { LOG_DEBUG(log, "Someone is already processing " << current_task_is_active_path); return PartitionTaskStatus::Active; @@ -1579,10 +1586,10 @@ protected: /// Try start processing, create node about it { String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); - auto op_create = std::make_shared(current_task_status_path, start_state, acl, zkutil::CreateMode::Persistent); + auto op_create = zkutil::makeCreateRequest(current_task_status_path, start_state, zkutil::CreateMode::Persistent); zkutil::MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create)); - if (info.code != ZOK) + if (info.code) { if (info.getFailedOp().getPath() == is_dirty_flag_path) { @@ -1721,10 +1728,10 @@ protected: /// Finalize the processing, change state of current partition task (and also check is_dirty flag) { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); - auto op_set = std::make_shared(current_task_status_path, state_finished, 0); + auto op_set = zkutil::makeSetRequest(current_task_status_path, state_finished, 0); zkutil::MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set)); - if (info.code != ZOK) + if (info.code) { if (info.getFailedOp().getPath() == is_dirty_flag_path) LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); diff --git a/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h b/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h index 35f56fa429f..a277eb220f1 100644 --- a/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h +++ b/dbms/src/Storages/MergeTree/AbandonableLockInZooKeeper.h @@ -30,7 +30,7 @@ public: }; AbandonableLockInZooKeeper( - const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Ops * precheck_ops = nullptr) + const String & path_prefix_, const String & temp_path, zkutil::ZooKeeper & zookeeper_, zkutil::Requests * precheck_ops = nullptr) : zookeeper(&zookeeper_), path_prefix(path_prefix_) { String abandonable_path = temp_path + "/abandonable_lock-"; @@ -42,12 +42,9 @@ public: } else { - precheck_ops->emplace_back(std::make_shared( - abandonable_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::EphemeralSequential)); - - zkutil::OpResultsPtr op_results = zookeeper->multi(*precheck_ops); - - holder_path = op_results->back().value; + precheck_ops->emplace_back(zkutil::makeCreateRequest(abandonable_path, "", zkutil::CreateMode::EphemeralSequential)); + zkutil::Responses op_results = zookeeper->multi(*precheck_ops); + holder_path = dynamic_cast(*op_results.back()).path_created; } /// Write the path to the secondary node in the main node. @@ -101,11 +98,11 @@ public: } /// Adds actions equivalent to `unlock()` to the list. - void getUnlockOps(zkutil::Ops & ops) + void getUnlockOps(zkutil::Requests & ops) { checkCreated(); - ops.emplace_back(std::make_shared(path, -1)); - ops.emplace_back(std::make_shared(holder_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(holder_path, -1)); } /// Do not delete nodes in destructor. You may call this method after 'getUnlockOps' and successful execution of these ops, @@ -128,7 +125,7 @@ public: try { - zookeeper->tryRemoveEphemeralNodeWithRetries(holder_path); + zookeeper->tryRemove(holder_path); zookeeper->trySet(path, ""); /// It's not necessary. } catch (...) @@ -156,7 +153,7 @@ public: /// If there is no secondary node, you need to test again the existence of the main node, /// because during this time you might have time to call unlock(). /// At the same time, we will remove the path to the secondary node from there. - if (zookeeper.trySet(path, "") == ZOK) + if (zookeeper.trySet(path, "") == ZooKeeperImpl::ZooKeeper::ZOK) return ABANDONED; return UNLOCKED; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp index f0513c04f9e..3c20b0a2069 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeAlterThread.cpp @@ -140,10 +140,10 @@ void ReplicatedMergeTreeAlterThread::run() ++changed_parts; /// Update part metadata in ZooKeeper. - zkutil::Ops ops; - ops.emplace_back(std::make_shared( + zkutil::Requests ops; + ops.emplace_back(zkutil::makeSetRequest( storage.replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1)); - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeSetRequest( storage.replica_path + "/parts/" + part->name + "/checksums", storage.getChecksumsForZooKeeper(transaction->getNewChecksums()), -1)); @@ -155,7 +155,7 @@ void ReplicatedMergeTreeAlterThread::run() catch (const zkutil::KeeperException & e) { /// The part does not exist in ZK. We will add to queue for verification - maybe the part is superfluous, and it must be removed locally. - if (e.code == ZNONODE) + if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE) storage.enqueuePartForCheck(part->name); throw; diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 6a77eec25bf..c165d04825f 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -205,21 +205,19 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// We remove the lock just after renaming the part. In case of exception, block number will be marked as abandoned. /// Also, make deduplication check. If a duplicate is detected, no nodes are created. - auto acl = zookeeper->getDefaultACL(); - /// Deduplication stuff bool deduplicate_block = !block_id.empty(); String block_id_path; - zkutil::Ops deduplication_check_ops; - zkutil::Ops * deduplication_check_ops_ptr = nullptr; + zkutil::Requests deduplication_check_ops; + zkutil::Requests * deduplication_check_ops_ptr = nullptr; if (deduplicate_block) { block_id_path = storage.zookeeper_path + "/blocks/" + block_id; /// Lets check for duplicates in advance, to avoid superflous block numbers allocation - deduplication_check_ops.emplace_back(std::make_shared(block_id_path, "", acl, zkutil::CreateMode::Persistent)); - deduplication_check_ops.emplace_back(std::make_shared(block_id_path, -1)); + deduplication_check_ops.emplace_back(zkutil::makeCreateRequest(block_id_path, "", zkutil::CreateMode::Persistent)); + deduplication_check_ops.emplace_back(zkutil::makeRemoveRequest(block_id_path, -1)); deduplication_check_ops_ptr = &deduplication_check_ops; } @@ -233,7 +231,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo { zkutil::MultiTransactionInfo & info = e.info; - if (deduplicate_block && info.code == ZNODEEXISTS && info.getFailedOp().getPath() == block_id_path) + if (deduplicate_block && info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && info.getFailedOp().getPath() == block_id_path) { LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (skip the insertion)"); part->is_duplicate = true; @@ -276,13 +274,13 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// Simultaneously add information about the part to all the necessary places in ZooKeeper and remove block_number_lock. /// Information about the part. - zkutil::Ops ops; + zkutil::Requests ops; if (deduplicate_block) { /// Make final duplicate check and commit block_id ops.emplace_back( - std::make_shared( + zkutil::makeCreateRequest( block_id_path, toString(block_number), /// We will able to know original part number for duplicate blocks, if we want. acl, @@ -291,30 +289,26 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// Information about the part, in the replica data. - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCheckRequest( storage.zookeeper_path + "/columns", storage.columns_version)); - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCreateRequest( storage.replica_path + "/parts/" + part->name, "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCreateRequest( storage.replica_path + "/parts/" + part->name + "/columns", part->columns.toString(), - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCreateRequest( storage.replica_path + "/parts/" + part->name + "/checksums", storage.getChecksumsForZooKeeper(part->checksums), - acl, zkutil::CreateMode::Persistent)); /// Replication log. - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCreateRequest( storage.zookeeper_path + "/log/log-", log_entry.toString(), - acl, zkutil::CreateMode::PersistentSequential)); /// Deletes the information that the block number is used for writing. @@ -339,7 +333,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo */ ops.emplace_back( - std::make_shared( + zkutil::makeCreateRequest( quorum_info.status_path, quorum_entry.toString(), acl, @@ -347,7 +341,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// Make sure that during the insertion time, the replica was not reinitialized or disabled (when the server is finished). ops.emplace_back( - std::make_shared( + zkutil::makeCheckRequest( storage.replica_path + "/is_active", quorum_info.is_active_node_version)); @@ -355,7 +349,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// But then the `host` value will change. We will check this. /// It's great that these two nodes change in the same transaction (see MergeTreeRestartingThread). ops.emplace_back( - std::make_shared( + zkutil::makeCheckRequest( storage.replica_path + "/host", quorum_info.host_node_version)); } @@ -366,7 +360,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo zkutil::MultiTransactionInfo info; zookeeper->tryMultiNoThrow(ops, nullptr, &info); /// 1 RTT - if (info.code == ZOK) + if (info.code == ZooKeeperImpl::ZooKeeper::ZOK) { transaction.commit(); storage.merge_selecting_event.set(); @@ -378,7 +372,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo { String failed_op_path = info.getFailedOp().getPath(); - if (info.code == ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) + if (info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) { /// Block with the same id have just appeared in table (or other replica), rollback thee insertion. LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")"); @@ -388,7 +382,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo last_block_is_duplicate = true; ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); } - else if (info.code == ZNODEEXISTS && failed_op_path == quorum_info.status_path) + else if (info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { transaction.rollback(); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 0352374fd0f..d6fdf64980a 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -95,15 +95,15 @@ void ReplicatedMergeTreeCleanupThread::clearOldLogs() if (entries.empty()) return; - zkutil::Ops ops; + zkutil::Requests ops; for (size_t i = 0; i < entries.size(); ++i) { - ops.emplace_back(std::make_shared(storage.zookeeper_path + "/log/" + entries[i], -1)); + ops.emplace_back(zkutil::makeRemoveRequest(storage.zookeeper_path + "/log/" + entries[i], -1)); if (ops.size() > 4 * zkutil::MULTI_BATCH_SIZE || i + 1 == entries.size()) { /// Simultaneously with clearing the log, we check to see if replica was added since we received replicas list. - ops.emplace_back(std::make_shared(storage.zookeeper_path + "/replicas", stat.version)); + ops.emplace_back(zkutil::makeCheckRequest(storage.zookeeper_path + "/replicas", stat.version)); zookeeper->multi(ops); ops.clear(); } @@ -159,7 +159,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() { const String & path = pair.first; int32_t rc = pair.second.get(); - if (rc == ZNOTEMPTY) + if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) { /// Can happen if there are leftover block nodes with children created by previous server versions. zookeeper->removeRecursive(path); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index d1f20f69585..273970497b5 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -161,21 +161,21 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper( /// because we update times in ZooKeeper with unlocked mutex, while these times may change. /// Consider it unimportant (for a short time, ZK will have a slightly different time value). - zkutil::Ops ops; + zkutil::Requests ops; if (min_unprocessed_insert_time_changed) - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeSetRequest( replica_path + "/min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1)); if (max_processed_insert_time_changed) - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeSetRequest( replica_path + "/max_processed_insert_time", toString(*max_processed_insert_time_changed), -1)); if (!ops.empty()) { auto code = zookeeper->tryMulti(ops); - if (code != ZOK) + if (code) LOG_ERROR(log, "Couldn't set value of nodes for insert times (" << replica_path << "/min_unprocessed_insert_time, max_processed_insert_time)" << ": " << zkutil::ZooKeeper::error2string(code) + ". This shouldn't happen often."); @@ -187,7 +187,7 @@ void ReplicatedMergeTreeQueue::remove(zkutil::ZooKeeperPtr zookeeper, LogEntryPt { auto code = zookeeper->tryRemove(replica_path + "/queue/" + entry->znode_name); - if (code != ZOK) + if (code) LOG_ERROR(log, "Couldn't remove " << replica_path << "/queue/" << entry->znode_name << ": " << zkutil::ZooKeeper::error2string(code) << ". This shouldn't happen often."); @@ -316,7 +316,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z /// Simultaneously add all new entries to the queue and move the pointer to the log. - zkutil::Ops ops; + zkutil::Requests ops; std::vector copied_entries; copied_entries.reserve(end - begin); @@ -327,8 +327,8 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z zkutil::ZooKeeper::ValueAndStat res = future.second.get(); copied_entries.emplace_back(LogEntry::parse(res.value, res.stat)); - ops.emplace_back(std::make_shared( - replica_path + "/queue/queue-", res.value, zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeCreateRequest( + replica_path + "/queue/queue-", res.value, zkutil::CreateMode::PersistentSequential)); const auto & entry = *copied_entries.back(); if (entry.type == LogEntry::GET_PART) @@ -342,14 +342,14 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z } } - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeSetRequest( replica_path + "/log_pointer", toString(last_entry_index + 1), -1)); if (min_unprocessed_insert_time_changed) - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeSetRequest( replica_path + "/min_unprocessed_insert_time", toString(*min_unprocessed_insert_time_changed), -1)); - zookeeper->multi(ops); + auto responses = zookeeper->multi(ops); /// Now we have successfully updated the queue in ZooKeeper. Update it in RAM. @@ -359,7 +359,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z for (size_t i = 0, size = copied_entries.size(); i < size; ++i) { - String path_created = dynamic_cast(*ops[i]).getPathCreated(); + String path_created = dynamic_cast(*responses[i]).path_created; copied_entries[i]->znode_name = path_created.substr(path_created.find_last_of('/') + 1); std::optional unused = false; @@ -451,7 +451,7 @@ void ReplicatedMergeTreeQueue::removeGetsAndMergesInRange(zkutil::ZooKeeperPtr z if ((*it)->currently_executing) to_wait.push_back(*it); auto code = zookeeper->tryRemove(replica_path + "/queue/" + (*it)->znode_name); - if (code != ZOK) + if (code) LOG_INFO(log, "Couldn't remove " << replica_path + "/queue/" + (*it)->znode_name << ": " << zkutil::ZooKeeper::error2string(code)); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index f8ea02cff9b..a3b4c9f2fee 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -260,10 +260,10 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts() { LOG_DEBUG(log, "Found part " << part_name << " with failed quorum. Moving to detached. This shouldn't happen often."); - zkutil::Ops ops; + zkutil::Requests ops; storage.removePartFromZooKeeper(part_name, ops); auto code = zookeeper->tryMulti(ops); - if (code == ZNONODE) + if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) LOG_WARNING(log, "Part " << part_name << " with failed quorum is not in ZooKeeper. This shouldn't happen often."); storage.data.renameAndDetachPart(part, "noquorum"); @@ -318,20 +318,19 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() { auto code = zookeeper->tryRemove(is_active_path, stat.version); - if (code == ZBADVERSION) + if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) throw Exception("Another instance of replica " + storage.replica_path + " was created just now." " You shouldn't run multiple instances of same replica. You need to check configuration files.", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE); - if (code != ZOK && code != ZNONODE) + if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE) throw zkutil::KeeperException(code, is_active_path); } /// Simultaneously declare that this replica is active, and update the host. - zkutil::Ops ops; - ops.emplace_back(std::make_shared(is_active_path, - active_node_identifier, zookeeper->getDefaultACL(), zkutil::CreateMode::Ephemeral)); - ops.emplace_back(std::make_shared(storage.replica_path + "/host", address.toString(), -1)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(is_active_path, active_node_identifier, zkutil::CreateMode::Ephemeral)); + ops.emplace_back(zkutil::makeSetRequest(storage.replica_path + "/host", address.toString(), -1)); try { @@ -339,7 +338,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() } catch (const zkutil::KeeperException & e) { - if (e.code == ZNODEEXISTS) + if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw Exception("Replica " + storage.replica_path + " appears to be already active. If you're sure it's not, " "try again in a minute or remove znode " + storage.replica_path + "/is_active manually", ErrorCodes::REPLICA_IS_ALREADY_ACTIVE); diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index e239ae3d65e..cb3e1f3061a 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -192,9 +192,14 @@ BlockInputStreams StorageDistributed::read( size_t num_remote_shards = cluster->getRemoteShardCount(); size_t result_size = (num_remote_shards * settings.max_parallel_replicas) + num_local_shards; - processed_stage = result_size == 1 || settings.distributed_group_by_no_merge - ? QueryProcessingStage::Complete - : QueryProcessingStage::WithMergeableState; + if (settings.distributed_group_by_no_merge) + processed_stage = QueryProcessingStage::Complete; + else if (settings.distributed_group_by_force_mergeable_state) + processed_stage = QueryProcessingStage::WithMergeableState; + else /// Normal mode. + processed_stage = result_size == 1 + ? QueryProcessingStage::Complete + : QueryProcessingStage::WithMergeableState; const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index f11d9fe28b8..d34077bfb00 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -477,32 +477,30 @@ void StorageReplicatedMergeTree::createTableIfNotExists() /// We write metadata of table so that the replicas can check table parameters with them. String metadata = TableMetadata(data).toString(); - auto acl = zookeeper->getDefaultACL(); - - zkutil::Ops ops; - ops.emplace_back(std::make_shared(zookeeper_path, "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/metadata", metadata, - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_unique(zookeeper_path + "/columns", getColumns().toString(), - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/log", "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/blocks", "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/block_numbers", "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/nonincrement_block_numbers", "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/leader_election", "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/temp", "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(zookeeper_path + "/replicas", "", - acl, zkutil::CreateMode::Persistent)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path, "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/metadata", metadata, + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/columns", getColumns().toString(), + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/log", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/blocks", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/block_numbers", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/nonincrement_block_numbers", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/leader_election", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/temp", "", + zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "", + zkutil::CreateMode::Persistent)); auto code = zookeeper->tryMulti(ops); - if (code != ZOK && code != ZNODEEXISTS) + if (code && code != ZNODEEXISTS) throw zkutil::KeeperException(code); } @@ -571,14 +569,13 @@ void StorageReplicatedMergeTree::createReplica() LOG_DEBUG(log, "Creating replica " << replica_path); /// Create an empty replica. We'll create `columns` node at the end - we'll use it as a sign that replica creation is complete. - auto acl = zookeeper->getDefaultACL(); - zkutil::Ops ops; - ops.emplace_back(std::make_shared(replica_path, "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(replica_path + "/host", "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(replica_path + "/log_pointer", "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(replica_path + "/queue", "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(replica_path + "/parts", "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared(replica_path + "/flags", "", acl, zkutil::CreateMode::Persistent)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest(replica_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/host", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/log_pointer", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/queue", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/parts", "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest(replica_path + "/flags", "", zkutil::CreateMode::Persistent)); try { @@ -586,7 +583,7 @@ void StorageReplicatedMergeTree::createReplica() } catch (const zkutil::KeeperException & e) { - if (e.code == ZNODEEXISTS) + if (e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw Exception("Replica " + replica_path + " already exists.", ErrorCodes::REPLICA_IS_ALREADY_EXIST); throw; @@ -837,7 +834,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) { LOG_ERROR(log, "Adding unexpected local part to ZooKeeper: " << part->name); - zkutil::Ops ops; + zkutil::Requests ops; checkPartChecksumsAndAddCommitOps(zookeeper, part, ops); zookeeper->multi(ops); } @@ -862,10 +859,10 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) log_entry.create_time = tryGetPartCreateTime(zookeeper, replica_path, name); /// We assume that this occurs before the queue is loaded (queue.initialize). - zkutil::Ops ops; + zkutil::Requests ops; removePartFromZooKeeper(name, ops); - ops.emplace_back(std::make_shared( - replica_path + "/queue/queue-", log_entry.toString(), zookeeper->getDefaultACL(), zkutil::CreateMode::PersistentSequential)); + ops.emplace_back(zkutil::makeCreateRequest( + replica_path + "/queue/queue-", log_entry.toString(), zkutil::CreateMode::PersistentSequential)); zookeeper->multi(ops); } @@ -879,7 +876,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, - const MergeTreeData::DataPartPtr & part, zkutil::Ops & ops, String part_name, NameSet * absent_replicas_paths) + const MergeTreeData::DataPartPtr & part, zkutil::Requests & ops, String part_name, NameSet * absent_replicas_paths) { if (part_name.empty()) part_name = part->name; @@ -934,17 +931,16 @@ void StorageReplicatedMergeTree::checkPartChecksumsAndAddCommitOps(const zkutil: if (!has_been_alredy_added) { - auto acl = zookeeper->getDefaultACL(); String part_path = replica_path + "/parts/" + part_name; - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCheckRequest( zookeeper_path + "/columns", expected_columns_version)); - ops.emplace_back(std::make_shared( - part_path, "", acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared( - part_path + "/columns", part->columns.toString(), acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared( - part_path + "/checksums", getChecksumsForZooKeeper(part->checksums), acl, zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest( + part_path, "", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest( + part_path + "/columns", part->columns.toString(), zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeCreateRequest( + part_path + "/checksums", getChecksumsForZooKeeper(part->checksums), zkutil::CreateMode::Persistent)); } else { @@ -960,7 +956,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd while (true) { - zkutil::Ops ops; + zkutil::Requests ops; NameSet absent_part_paths_on_replicas; /// Checksums are checked here and `ops` is filled. In fact, the part is added to ZK just below, when executing `multi`. @@ -973,11 +969,11 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd /// Will check that the part did not suddenly appear on skipped replicas if (!absent_part_paths_on_replicas.empty()) { - zkutil::Ops new_ops; + zkutil::Requests new_ops; for (const String & part_path : absent_part_paths_on_replicas) { - new_ops.emplace_back(std::make_shared(part_path, "", zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent)); - new_ops.emplace_back(std::make_shared(part_path, -1)); + new_ops.emplace_back(zkutil::makeCreateRequest(part_path, "", zkutil::CreateMode::Persistent)); + new_ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1)); } /// Add check ops at the beginning @@ -995,7 +991,7 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd size_t num_check_ops = 2 * absent_part_paths_on_replicas.size(); size_t failed_op_index = zkutil::getFailedOpIndex(e.info.op_results, e.info.code); - if (failed_op_index < num_check_ops && e.info.code == ZNODEEXISTS) + if (failed_op_index < num_check_ops && e.info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { LOG_INFO(log, "The part " << e.info.getFailedOp().describe() << " on a replica suddenly appeared, will recheck checksums"); } @@ -1347,14 +1343,14 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree:: Strings replicas = zookeeper->getChildren(zookeeper_path + "/replicas"); - zkutil::Ops ops; + zkutil::Requests ops; for (size_t i = 0, size = replicas.size(); i < size; ++i) { Stat stat; String path = zookeeper_path + "/replicas/" + replicas[i] + "/host"; zookeeper->get(path, &stat); - ops.emplace_back(std::make_shared(path, stat.version)); + ops.emplace_back(zkutil::makeCheckRequest(path, stat.version)); } /// We verify that while we were collecting versions, the replica with the necessary part did not come alive. @@ -1373,7 +1369,7 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree:: if (quorum_entry.part_name == entry.new_part_name) { - ops.emplace_back(std::make_shared(quorum_path, quorum_stat.version)); + ops.emplace_back(zkutil::makeRemoveRequest(quorum_path, quorum_stat.version)); auto part_info = MergeTreePartInfo::fromPartName(entry.new_part_name, data.format_version); @@ -1383,32 +1379,28 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree:: zookeeper->createIfNotExists(zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id, ""); - auto acl = zookeeper->getDefaultACL(); - - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/nonincrement_block_numbers/" + part_info.partition_id + "/block-" + padIndex(part_info.min_block), "", - acl, zkutil::CreateMode::Persistent)); - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeCreateRequest( zookeeper_path + "/quorum/failed_parts/" + entry.new_part_name, "", - acl, zkutil::CreateMode::Persistent)); /// Deleting from `blocks`. if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id)) - ops.emplace_back(std::make_shared(zookeeper_path + "/blocks/" + entry.block_id, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1)); auto code = zookeeper->tryMulti(ops); - if (code == ZOK) + if (code == ZooKeeperImpl::ZooKeeper::ZOK) { LOG_DEBUG(log, "Marked quorum for part " << entry.new_part_name << " as failed."); return true; /// NOTE Deletion from `virtual_parts` is not done, but it is only necessary for merges. } - else if (code == ZBADVERSION || code == ZNONODE || code == ZNODEEXISTS) + else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part " << entry.new_part_name << " as failed. Code: " << zerror(code)); @@ -1514,13 +1506,13 @@ void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTr if (entry.detach) data.renameAndDetachPart(part); - zkutil::Ops ops; + zkutil::Requests ops; removePartFromZooKeeper(part->name, ops); auto code = getZooKeeper()->tryMulti(ops); /// If the part is already removed (for example, because it was never added to ZK due to crash, /// see ReplicatedMergeTreeBlockOutputStream), then Ok. - if (code != ZOK && code != ZNONODE) + if (code && code != ZooKeeperImpl::ZooKeeper::ZNONODE) throw zkutil::KeeperException(code); /// If the part needs to be removed, it is more reliable to delete the directory after the changes in ZooKeeper. @@ -1571,10 +1563,10 @@ void StorageReplicatedMergeTree::executeClearColumnInPartition(const LogEntry & continue; /// Update part metadata in ZooKeeper. - zkutil::Ops ops; - ops.emplace_back(std::make_shared( + zkutil::Requests ops; + ops.emplace_back(zkutil::makeSetRequest( replica_path + "/parts/" + part->name + "/columns", transaction->getNewColumns().toString(), -1)); - ops.emplace_back(std::make_shared( + ops.emplace_back(zkutil::makeSetRequest( replica_path + "/parts/" + part->name + "/checksums", getChecksumsForZooKeeper(transaction->getNewChecksums()), -1)); zookeeper->multi(ops); @@ -2013,13 +2005,13 @@ bool StorageReplicatedMergeTree::createLogEntryToMergeParts( } -void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops) +void StorageReplicatedMergeTree::removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops) { String part_path = replica_path + "/parts/" + part_name; - ops.emplace_back(std::make_shared(part_path + "/checksums", -1)); - ops.emplace_back(std::make_shared(part_path + "/columns", -1)); - ops.emplace_back(std::make_shared(part_path, -1)); + ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/checksums", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(part_path + "/columns", -1)); + ops.emplace_back(zkutil::makeRemoveRequest(part_path, -1)); } @@ -2035,16 +2027,16 @@ void StorageReplicatedMergeTree::removePartAndEnqueueFetch(const String & part_n log_entry->source_replica = ""; log_entry->new_part_name = part_name; - zkutil::Ops ops; - ops.emplace_back(std::make_shared( - replica_path + "/queue/queue-", log_entry->toString(), zookeeper->getDefaultACL(), + zkutil::Requests ops; + ops.emplace_back(zkutil::makeCreateRequest( + replica_path + "/queue/queue-", log_entry->toString(), zkutil::CreateMode::PersistentSequential)); removePartFromZooKeeper(part_name, ops); auto results = zookeeper->multi(ops); - String path_created = dynamic_cast(*ops[0]).getPathCreated(); + String path_created = dynamic_cast(*results[0]).path_created; log_entry->znode_name = path_created.substr(path_created.find_last_of('/') + 1); queue.insert(zookeeper, log_entry); } @@ -2185,21 +2177,21 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) { /// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum. - zkutil::Ops ops; - ops.emplace_back(std::make_shared(quorum_status_path, stat.version)); - ops.emplace_back(std::make_shared(quorum_last_part_path, part_name, -1)); + zkutil::Requests ops; + ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); + ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1)); auto code = zookeeper->tryMulti(ops); - if (code == ZOK) + if (code == ZooKeeperImpl::ZooKeeper::ZOK) { break; } - else if (code == ZNONODE) + else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) { /// The quorum has already been achieved. break; } - else if (code == ZBADVERSION) + else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) { /// Node was updated meanwhile. We must re-read it and repeat all the actions. continue; @@ -2212,16 +2204,16 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) /// We update the node, registering there one more replica. auto code = zookeeper->trySet(quorum_status_path, quorum_entry.toString(), stat.version); - if (code == ZOK) + if (code == ZooKeeperImpl::ZooKeeper::ZOK) { break; } - else if (code == ZNONODE) + else if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) { /// The quorum has already been achieved. break; } - else if (code == ZBADVERSION) + else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION) { /// Node was updated meanwhile. We must re-read it and repeat all the actions. continue; @@ -2975,7 +2967,7 @@ void StorageReplicatedMergeTree::drop() /// Check that `zookeeper_path` exists: it could have been deleted by another replica after execution of previous line. Strings replicas; - if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZOK && replicas.empty()) + if (zookeeper->tryGetChildren(zookeeper_path + "/replicas", replicas) == ZooKeeperImpl::ZooKeeper::ZOK && replicas.empty()) { LOG_INFO(log, "Removing table " << zookeeper_path << " (this might take several minutes)"); zookeeper->tryRemoveRecursive(zookeeper_path); @@ -3021,13 +3013,13 @@ bool StorageReplicatedMergeTree::existsNodeCached(const std::string & path) AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, - zkutil::Ops * precheck_ops) + zkutil::Requests * precheck_ops) { String partition_path = zookeeper_path + "/block_numbers/" + partition_id; if (!existsNodeCached(partition_path)) { int code = zookeeper->tryCreate(partition_path, "", zkutil::CreateMode::Persistent); - if (code != ZOK && code != ZNODEEXISTS) + if (code && code != ZNODEEXISTS) throw zkutil::KeeperException(code, partition_path); } @@ -3648,12 +3640,12 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() } -static int32_t tryMultiWithRetries(zkutil::ZooKeeperPtr & zookeeper, zkutil::Ops & ops) noexcept +static int32_t tryMulti(zkutil::ZooKeeperPtr & zookeeper, zkutil::Requests & ops) noexcept { int32_t code; try { - code = zookeeper->tryMultiWithRetries(ops); + code = zookeeper->tryMulti(ops); } catch (const zkutil::KeeperException & e) { @@ -3667,7 +3659,7 @@ static int32_t tryMultiWithRetries(zkutil::ZooKeeperPtr & zookeeper, zkutil::Ops void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, NameSet * parts_should_be_retied) { - zkutil::Ops ops; + zkutil::Requests ops; auto it_first_node_in_batch = part_names.cbegin(); for (auto it = part_names.cbegin(); it != part_names.cend(); ++it) @@ -3678,21 +3670,21 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend()) { /// It is Ok to use multi with retries to delete nodes, because new nodes with the same names cannot appear here - auto code = tryMultiWithRetries(zookeeper, ops); + auto code = tryMulti(zookeeper, ops); ops.clear(); - if (code == ZNONODE) + if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) { /// Fallback LOG_DEBUG(log, "There are no some part nodes in ZooKeeper, will remove part nodes sequentially"); for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch) { - zkutil::Ops cur_ops; + zkutil::Requests cur_ops; removePartFromZooKeeper(*it_in_batch, cur_ops); - auto cur_code = tryMultiWithRetries(zookeeper, cur_ops); + auto cur_code = tryMulti(zookeeper, cur_ops); - if (cur_code == ZNONODE) + if (cur_code == ZooKeeperImpl::ZooKeeper::ZNONODE) { LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem"); } @@ -3700,7 +3692,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & { parts_should_be_retied->emplace(*it_in_batch); } - else if (cur_code != ZOK) + else if (cur_code) { LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << ::zerror(cur_code)); } @@ -3711,7 +3703,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch) parts_should_be_retied->emplace(*it_in_batch); } - else if (code != ZOK) + else if (code) { LOG_WARNING(log, "There was a problem with deleting " << (it_next - it_first_node_in_batch) << " nodes from ZooKeeper: " << ::zerror(code)); @@ -3761,7 +3753,7 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( { const String & path = pair.first; int32_t rc = pair.second.get(); - if (rc == ZNOTEMPTY) + if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) { /// Can happen if there are leftover block nodes with children created by previous server versions. zookeeper.removeRecursive(path); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index b16e2538fe5..c2b09a77bf1 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -321,7 +321,7 @@ private: * Call under TableStructureLock. */ void checkPartChecksumsAndAddCommitOps(const zkutil::ZooKeeperPtr & zookeeper, const MergeTreeData::DataPartPtr & part, - zkutil::Ops & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr); + zkutil::Requests & ops, String part_name = "", NameSet * absent_replicas_paths = nullptr); String getChecksumsForZooKeeper(const MergeTreeDataPartChecksums & checksums); @@ -330,7 +330,7 @@ private: const MergeTreeData::DataPartPtr & part); /// Adds actions to `ops` that remove a part from ZooKeeper. - void removePartFromZooKeeper(const String & part_name, zkutil::Ops & ops); + void removePartFromZooKeeper(const String & part_name, zkutil::Requests & ops); /// Quickly removes big set of parts from ZooKeeper (using async multi queries) void removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, @@ -415,7 +415,7 @@ private: /// Creates new block number and additionally perform precheck_ops while creates 'abandoned node' AbandonableLockInZooKeeper allocateBlockNumber(const String & partition_id, zkutil::ZooKeeperPtr & zookeeper, - zkutil::Ops * precheck_ops = nullptr); + zkutil::Requests * precheck_ops = nullptr); /** Wait until all replicas, including this, execute the specified action from the log. * If replicas are added at the same time, it can not wait the added replica . From 26584844b60e304ce109fe86c12769cb1f0b22ec Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Mar 2018 04:00:12 +0300 Subject: [PATCH 36/70] Rewriting ZooKeeper library [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/Types.h | 2 +- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 3 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 16 +++---- .../ReplicatedMergeTreeCleanupThread.cpp | 6 +-- .../ReplicatedMergeTreeCleanupThread.h | 1 + .../Storages/StorageReplicatedMergeTree.cpp | 46 ++++++++----------- 6 files changed, 33 insertions(+), 41 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/Types.h b/dbms/src/Common/ZooKeeper/Types.h index 05419f623ef..5ccfc40e011 100644 --- a/dbms/src/Common/ZooKeeper/Types.h +++ b/dbms/src/Common/ZooKeeper/Types.h @@ -53,7 +53,7 @@ using CheckResponse = ZooKeeperImpl::ZooKeeper::CheckResponse; RequestPtr makeCreateRequest(const std::string & path, const std::string & data, int create_mode); RequestPtr makeRemoveRequest(const std::string & path, int version); -RequestPtr makeSetRequest(const std::string & path, const std::string & data); +RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version); RequestPtr makeCheckRequest(const std::string & path, int version); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index e4bf8b2f936..fef461b24df 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -849,11 +849,12 @@ RequestPtr makeRemoveRequest(const std::string & path, int version) return request; } -RequestPtr makeSetRequest(const std::string & path, const std::string & data) +RequestPtr makeSetRequest(const std::string & path, const std::string & data, int version) { auto request = std::make_shared(); request->path = path; request->data = data; + request->version = version; return request; } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index c165d04825f..5ab143119d5 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -60,9 +60,9 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe { quorum_info.status_path = storage.zookeeper_path + "/quorum/status"; - zkutil::ZooKeeper::TryGetFuture quorum_status_future = zookeeper->asyncTryGet(quorum_info.status_path); - zkutil::ZooKeeper::TryGetFuture is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active"); - zkutil::ZooKeeper::TryGetFuture host_future = zookeeper->asyncTryGet(storage.replica_path + "/host"); + std::future quorum_status_future = zookeeper->asyncTryGet(quorum_info.status_path); + std::future is_active_future = zookeeper->asyncTryGet(storage.replica_path + "/is_active"); + std::future host_future = zookeeper->asyncTryGet(storage.replica_path + "/host"); /// List of live replicas. All of them register an ephemeral node for leader_election. @@ -83,18 +83,18 @@ void ReplicatedMergeTreeBlockOutputStream::checkQuorumPrecondition(zkutil::ZooKe */ auto quorum_status = quorum_status_future.get(); - if (quorum_status.exists) - throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.value, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); + if (quorum_status.error != ZooKeeperImpl::ZooKeeper::ZNONODE) + throw Exception("Quorum for previous write has not been satisfied yet. Status: " + quorum_status.data, ErrorCodes::UNSATISFIED_QUORUM_FOR_PREVIOUS_WRITE); /// Both checks are implicitly made also later (otherwise there would be a race condition). auto is_active = is_active_future.get(); auto host = host_future.get(); - if (!is_active.exists || !host.exists) + if (is_active.error == ZooKeeperImpl::ZooKeeper::ZNONODE || host.error == ZooKeeperImpl::ZooKeeper::ZNONODE) throw Exception("Replica is not active right now", ErrorCodes::READONLY); - quorum_info.is_active_node_value = is_active.value; + quorum_info.is_active_node_value = is_active.data; quorum_info.is_active_node_version = is_active.stat.version; quorum_info.host_node_version = host.stat.version; } @@ -283,7 +283,6 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo zkutil::makeCreateRequest( block_id_path, toString(block_number), /// We will able to know original part number for duplicate blocks, if we want. - acl, zkutil::CreateMode::Persistent)); } @@ -336,7 +335,6 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo zkutil::makeCreateRequest( quorum_info.status_path, quorum_entry.toString(), - acl, zkutil::CreateMode::Persistent)); /// Make sure that during the insertion time, the replica was not reinitialized or disabled (when the server is finished). diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index d6fdf64980a..8a1069f38b3 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -148,7 +148,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime); auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold); - std::vector> try_remove_futures; + std::vector> try_remove_futures; for (auto it = first_outdated_block; it != timed_blocks.end(); ++it) { String path = storage.zookeeper_path + "/blocks/" + it->node; @@ -164,7 +164,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() /// Can happen if there are leftover block nodes with children created by previous server versions. zookeeper->removeRecursive(path); } - else if (rc != ZOK) + else if (rc) LOG_WARNING(log, "Error while deleting ZooKeeper path `" << path << "`: " + zkutil::ZooKeeper::error2string(rc) << ", ignoring."); } @@ -181,7 +181,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper & Strings blocks; zkutil::Stat stat; - if (ZOK != zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat)) + if (zookeeper.tryGetChildren(storage.zookeeper_path + "/blocks", blocks, &stat)) throw Exception(storage.zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); /// Clear already deleted blocks from the cache, cached_block_ctime should be subset of blocks diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h index b9fbda531a9..c717e84bfd5 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index d34077bfb00..6e00343cc02 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -232,7 +232,7 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree( catch (const zkutil::KeeperException & e) { /// Failed to connect to ZK (this became known when trying to perform the first operation). - if (e.code == ZCONNECTIONLOSS) + if (e.code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS) { tryLogCurrentException(__PRETTY_FUNCTION__); current_zookeeper = nullptr; @@ -500,7 +500,7 @@ void StorageReplicatedMergeTree::createTableIfNotExists() zkutil::CreateMode::Persistent)); auto code = zookeeper->tryMulti(ops); - if (code && code != ZNODEEXISTS) + if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw zkutil::KeeperException(code); } @@ -601,7 +601,7 @@ void StorageReplicatedMergeTree::createReplica() */ String source_replica; - Stat stat; + zkutil::Stat stat; zookeeper->exists(replica_path, &stat); auto my_create_time = stat.czxid; @@ -1347,7 +1347,7 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree:: for (size_t i = 0, size = replicas.size(); i < size; ++i) { - Stat stat; + zkutil::Stat stat; String path = zookeeper_path + "/replicas/" + replicas[i] + "/host"; zookeeper->get(path, &stat); ops.emplace_back(zkutil::makeCheckRequest(path, stat.version)); @@ -1361,7 +1361,7 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree:: if (replica.empty()) { - Stat quorum_stat; + zkutil::Stat quorum_stat; String quorum_path = zookeeper_path + "/quorum/status"; String quorum_str = zookeeper->get(quorum_path, &quorum_stat); ReplicatedMergeTreeQuorumEntry quorum_entry; @@ -1403,7 +1403,7 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree:: else if (code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || code == ZooKeeperImpl::ZooKeeper::ZNONODE || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { LOG_DEBUG(log, "State was changed or isn't expected when trying to mark quorum for part " - << entry.new_part_name << " as failed. Code: " << zerror(code)); + << entry.new_part_name << " as failed. Code: " << zkutil::ZooKeeper::error2string(code)); } else throw zkutil::KeeperException(code); @@ -1601,14 +1601,6 @@ void StorageReplicatedMergeTree::queueUpdatingThread() update_in_progress = false; queue_updating_event->wait(); } - catch (const zkutil::KeeperException & e) - { - if (e.code == ZINVALIDSTATE) - restarting_thread->wakeup(); - - tryLogCurrentException(__PRETTY_FUNCTION__); - queue_updating_event->tryWait(QUEUE_UPDATE_ERROR_SLEEP_MS); - } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); @@ -3019,7 +3011,7 @@ AbandonableLockInZooKeeper StorageReplicatedMergeTree::allocateBlockNumber(const if (!existsNodeCached(partition_path)) { int code = zookeeper->tryCreate(partition_path, "", zkutil::CreateMode::Persistent); - if (code && code != ZNODEEXISTS) + if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw zkutil::KeeperException(code, partition_path); } @@ -3688,17 +3680,17 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & { LOG_DEBUG(log, "There is no part " << *it_in_batch << " in ZooKeeper, it was only in filesystem"); } - else if (parts_should_be_retied && zkutil::isHardwareErrorCode(cur_code)) + else if (parts_should_be_retied && zkutil::isHardwareError(cur_code)) { parts_should_be_retied->emplace(*it_in_batch); } else if (cur_code) { - LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << ::zerror(cur_code)); + LOG_WARNING(log, "Cannot remove part " << *it_in_batch << " from ZooKeeper: " << zkutil::ZooKeeper::error2string(cur_code)); } } } - else if (parts_should_be_retied && zkutil::isHardwareErrorCode(code)) + else if (parts_should_be_retied && zkutil::isHardwareError(code)) { for (auto it_in_batch = it_first_node_in_batch; it_in_batch != it_next; ++it_in_batch) parts_should_be_retied->emplace(*it_in_batch); @@ -3706,7 +3698,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & else if (code) { LOG_WARNING(log, "There was a problem with deleting " << (it_next - it_first_node_in_batch) - << " nodes from ZooKeeper: " << ::zerror(code)); + << " nodes from ZooKeeper: " << ::zkutil::ZooKeeper::error2string(code)); } it_first_node_in_batch = it_next; @@ -3719,11 +3711,11 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( zkutil::ZooKeeper & zookeeper, const String & partition_id, Int64 min_block_num, Int64 max_block_num) { Strings blocks; - if (ZOK != zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks)) + if (zookeeper.tryGetChildren(zookeeper_path + "/blocks", blocks)) throw Exception(zookeeper_path + "/blocks doesn't exist", ErrorCodes::NOT_FOUND_NODE); String partition_prefix = partition_id + "_"; - std::vector> get_futures; + std::vector>> get_futures; for (const String & block_id : blocks) { if (startsWith(block_id, partition_prefix)) @@ -3733,16 +3725,16 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( } } - std::vector> to_delete_futures; + std::vector>> to_delete_futures; for (auto & pair : get_futures) { const String & path = pair.first; - zkutil::ZooKeeper::ValueAndStatAndExists result = pair.second.get(); + auto result = pair.second.get(); - if (!result.exists) + if (result.error == ZooKeeperImpl::ZooKeeper::ZNONODE) continue; - ReadBufferFromString buf(result.value); + ReadBufferFromString buf(result.data); Int64 block_num = 0; bool parsed = tryReadIntText(block_num, buf) && buf.eof(); if (!parsed || (min_block_num <= block_num && block_num <= max_block_num)) @@ -3752,13 +3744,13 @@ void StorageReplicatedMergeTree::clearBlocksInPartition( for (auto & pair : to_delete_futures) { const String & path = pair.first; - int32_t rc = pair.second.get(); + int32_t rc = pair.second.get().error; if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) { /// Can happen if there are leftover block nodes with children created by previous server versions. zookeeper.removeRecursive(path); } - else if (rc != ZOK) + else if (rc) LOG_WARNING(log, "Error while deleting ZooKeeper path `" << path << "`: " + zkutil::ZooKeeper::error2string(rc) << ", ignoring."); } From ae9633c1e58d30f196086682568eec3c45b3d6f3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Mar 2018 23:00:16 +0300 Subject: [PATCH 37/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/KeeperException.h | 4 ++- dbms/src/Common/ZooKeeper/Types.h | 3 ++ dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 8 ++++- .../ReplicatedMergeTreeBlockOutputStream.cpp | 30 +++++++++---------- .../ReplicatedMergeTreeCleanupThread.cpp | 10 +++---- .../MergeTree/ReplicatedMergeTreeLogEntry.h | 6 ++-- .../Storages/StorageReplicatedMergeTree.cpp | 8 ++--- 7 files changed, 38 insertions(+), 31 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/KeeperException.h b/dbms/src/Common/ZooKeeper/KeeperException.h index e188ba90d4d..79dce371a19 100644 --- a/dbms/src/Common/ZooKeeper/KeeperException.h +++ b/dbms/src/Common/ZooKeeper/KeeperException.h @@ -86,12 +86,14 @@ class KeeperMultiException : public KeeperException public: Requests requests; Responses responses; + size_t failed_op_index = 0; + + std::string getPathForFirstFailedOp() const; /// If it is user error throws KeeperMultiException else throws ordinary KeeperException /// If it is ZOK does nothing static void check(int32_t code, const Requests & requests, const Responses & responses); -protected: KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses); }; diff --git a/dbms/src/Common/ZooKeeper/Types.h b/dbms/src/Common/ZooKeeper/Types.h index 5ccfc40e011..5cbc3cdd24a 100644 --- a/dbms/src/Common/ZooKeeper/Types.h +++ b/dbms/src/Common/ZooKeeper/Types.h @@ -29,6 +29,9 @@ using EventPtr = std::shared_ptr; /// they must execute as quickly as possible (preferably just set some notification). using WatchCallback = ZooKeeperImpl::ZooKeeper::WatchCallback; +using Request = ZooKeeperImpl::ZooKeeper::Request; +using Response = ZooKeeperImpl::ZooKeeper::Response; + using RequestPtr = ZooKeeperImpl::ZooKeeper::RequestPtr; using ResponsePtr = ZooKeeperImpl::ZooKeeper::ResponsePtr; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index fef461b24df..be80588bce6 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -815,10 +815,16 @@ size_t getFailedOpIndex(const Responses & responses, int32_t transaction_return_ KeeperMultiException::KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses) : KeeperException("Transaction failed at op #" + std::to_string(getFailedOpIndex(responses, code)), code), - requests(requests), responses(responses) + requests(requests), responses(responses), failed_op_index(getFailedOpIndex(responses, code)) { } +std::string KeeperMultiException::getPathForFirstFailedOp() const +{ + return requests[failed_op_index]->getPath(); + +} + void KeeperMultiException::check(int32_t code, const Requests & requests, const Responses & responses) { if (!code) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index 5ab143119d5..9fafd5f2e9d 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -227,11 +227,9 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// 2 RTT block_number_lock = storage.allocateBlockNumber(part->info.partition_id, zookeeper, deduplication_check_ops_ptr); } - catch (zkutil::KeeperMultiException & e) + catch (const zkutil::KeeperMultiException & e) { - zkutil::MultiTransactionInfo & info = e.info; - - if (deduplicate_block && info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && info.getFailedOp().getPath() == block_id_path) + if (deduplicate_block && e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && e.getPathForFirstFailedOp() == block_id_path) { LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (skip the insertion)"); part->is_duplicate = true; @@ -242,7 +240,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION); } - catch (zkutil::KeeperException & e) + catch (const zkutil::KeeperException & e) { throw Exception("Cannot allocate block number in ZooKeeper: " + e.displayText(), ErrorCodes::KEEPER_EXCEPTION); } @@ -355,10 +353,10 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo MergeTreeData::Transaction transaction; /// If you can not add a part to ZK, we'll remove it back from the working set. storage.data.renameTempPartAndAdd(part, nullptr, &transaction); - zkutil::MultiTransactionInfo info; - zookeeper->tryMultiNoThrow(ops, nullptr, &info); /// 1 RTT + zkutil::Responses responses; + int32_t multi_code = zookeeper->tryMultiNoThrow(ops, responses); /// 1 RTT - if (info.code == ZooKeeperImpl::ZooKeeper::ZOK) + if (multi_code == ZooKeeperImpl::ZooKeeper::ZOK) { transaction.commit(); storage.merge_selecting_event.set(); @@ -366,11 +364,11 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// Lock nodes have been already deleted, do not delete them in destructor block_number_lock.assumeUnlocked(); } - else if (zkutil::isUserError(info.code)) + else if (zkutil::isUserError(multi_code)) { - String failed_op_path = info.getFailedOp().getPath(); + String failed_op_path = zkutil::KeeperMultiException(multi_code, ops, responses).getPathForFirstFailedOp(); - if (info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) + if (multi_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && deduplicate_block && failed_op_path == block_id_path) { /// Block with the same id have just appeared in table (or other replica), rollback thee insertion. LOG_INFO(log, "Block with ID " << block_id << " already exists; ignoring it (removing part " << part->name << ")"); @@ -380,7 +378,7 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo last_block_is_duplicate = true; ProfileEvents::increment(ProfileEvents::DuplicatedInsertedBlocks); } - else if (info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && failed_op_path == quorum_info.status_path) + else if (multi_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS && failed_op_path == quorum_info.status_path) { transaction.rollback(); @@ -391,21 +389,21 @@ void ReplicatedMergeTreeBlockOutputStream::commitPart(zkutil::ZooKeeperPtr & zoo /// NOTE: We could be here if the node with the quorum existed, but was quickly removed. transaction.rollback(); throw Exception("Unexpected logical error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + zkutil::ZooKeeper::error2string(info.code) + ", path " + failed_op_path, + + zkutil::ZooKeeper::error2string(multi_code) + ", path " + failed_op_path, ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } } - else if (zkutil::isHardwareError(info.code)) + else if (zkutil::isHardwareError(multi_code)) { transaction.rollback(); throw Exception("Unrecoverable network error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + zkutil::ZooKeeper::error2string(info.code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + + zkutil::ZooKeeper::error2string(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } else { transaction.rollback(); throw Exception("Unexpected ZooKeeper error while adding block " + toString(block_number) + " with ID '" + block_id + "': " - + zkutil::ZooKeeper::error2string(info.code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); + + zkutil::ZooKeeper::error2string(multi_code), ErrorCodes::UNEXPECTED_ZOOKEEPER_ERROR); } if (quorum) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp index 8a1069f38b3..f7dca129bd3 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeCleanupThread.cpp @@ -148,7 +148,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() auto first_outdated_block_time_threshold = std::upper_bound(timed_blocks.begin(), timed_blocks.end(), block_threshold, NodeWithStat::greaterByTime); auto first_outdated_block = std::min(first_outdated_block_fixed_threshold, first_outdated_block_time_threshold); - std::vector> try_remove_futures; + std::vector>> try_remove_futures; for (auto it = first_outdated_block; it != timed_blocks.end(); ++it) { String path = storage.zookeeper_path + "/blocks/" + it->node; @@ -158,7 +158,7 @@ void ReplicatedMergeTreeCleanupThread::clearOldBlocks() for (auto & pair : try_remove_futures) { const String & path = pair.first; - int32_t rc = pair.second.get(); + int32_t rc = pair.second.get().error; if (rc == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) { /// Can happen if there are leftover block nodes with children created by previous server versions. @@ -203,7 +203,7 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper & << " to clear old ones from ZooKeeper."); } - std::vector> exists_futures; + std::vector>> exists_futures; for (const String & block : blocks) { auto it = cached_block_stats.find(block); @@ -222,8 +222,8 @@ void ReplicatedMergeTreeCleanupThread::getBlocksSortedByTime(zkutil::ZooKeeper & /// Put fetched stats into the cache for (auto & elem : exists_futures) { - zkutil::ZooKeeper::StatAndExists status = elem.second.get(); - if (status.exists) + auto status = elem.second.get(); + if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE) { cached_block_stats.emplace(elem.first, status.stat.ctime); timed_blocks.emplace_back(elem.first, status.stat.ctime); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h index 913115f185c..3bc76423f2b 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeLogEntry.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include @@ -8,9 +9,6 @@ #include -struct Stat; - - namespace DB { @@ -100,7 +98,7 @@ struct ReplicatedMergeTreeLogEntry : ReplicatedMergeTreeLogEntryData std::condition_variable execution_complete; /// Awake when currently_executing becomes false. - static Ptr parse(const String & s, const Stat & stat); + static Ptr parse(const String & s, const zkutil::Stat & stat); }; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 6e00343cc02..8a5f20b8421 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -986,14 +986,14 @@ MergeTreeData::DataPartsVector StorageReplicatedMergeTree::checkPartChecksumsAnd zookeeper->multi(ops); return transaction.commit(); } - catch (zkutil::KeeperMultiException & e) + catch (const zkutil::KeeperMultiException & e) { size_t num_check_ops = 2 * absent_part_paths_on_replicas.size(); - size_t failed_op_index = zkutil::getFailedOpIndex(e.info.op_results, e.info.code); + size_t failed_op_index = e.failed_op_index; - if (failed_op_index < num_check_ops && e.info.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) + if (failed_op_index < num_check_ops && e.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) { - LOG_INFO(log, "The part " << e.info.getFailedOp().describe() << " on a replica suddenly appeared, will recheck checksums"); + LOG_INFO(log, "The part " << e.getPathForFirstFailedOp() << " on a replica suddenly appeared, will recheck checksums"); } else throw; From 58f32f98d71af60146daad4e195e7ac5a26710ae Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Mar 2018 23:11:46 +0300 Subject: [PATCH 38/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- .../tests/gtest_zkutil_test_multi_exception.cpp | 2 +- .../Common/ZooKeeper/tests/zkutil_test_async.cpp | 2 +- dbms/src/Server/ClusterCopier.cpp | 4 ++-- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 16 +++++++++------- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- .../Storages/System/StorageSystemZooKeeper.cpp | 4 ++-- 6 files changed, 16 insertions(+), 14 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp index 2119f52e2c1..f4f0d0ac86d 100644 --- a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp +++ b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp @@ -93,7 +93,7 @@ TEST(zkutil, multi_async) EXPECT_ANY_THROW ( - std::vector futures; + std::vector> futures; for (size_t i = 0; i < 10000; ++i) { diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp index 7dd4a619373..8bf47ecf05c 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp @@ -23,7 +23,7 @@ try { while (true) { - std::vector futures; + std::vector> futures; for (auto & node : nodes) futures.push_back(zookeeper.asyncTryGet("/tmp/" + node)); diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index d46b91165f2..7f330d5de1f 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -1660,7 +1660,7 @@ protected: output = io_insert.out; } - using ExistsFuture = zkutil::ZooKeeper::ExistsFuture; + using ExistsFuture = std::future; auto future_is_dirty_checker = std::make_unique(zookeeper->asyncExists(is_dirty_flag_path)); Stopwatch watch(CLOCK_MONOTONIC_COARSE); @@ -1686,7 +1686,7 @@ protected: throw; } - if (status.exists) + if (status.error != ZooKeeperImpl::ZooKeeper::ZNONODE) throw Exception("Partition is dirty, cancel INSERT SELECT", ErrorCodes::UNFINISHED); } diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 273970497b5..9360b2c62e6 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -53,7 +53,7 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) std::sort(children.begin(), children.end()); - std::vector> futures; + std::vector>> futures; futures.reserve(children.size()); for (const String & child : children) @@ -61,8 +61,9 @@ bool ReplicatedMergeTreeQueue::load(zkutil::ZooKeeperPtr zookeeper) for (auto & future : futures) { - zkutil::ZooKeeper::ValueAndStat res = future.second.get(); - LogEntryPtr entry = LogEntry::parse(res.value, res.stat); + zkutil::GetResponse res = future.second.get(); + + LogEntryPtr entry = LogEntry::parse(res.data, res.stat); entry->znode_name = future.first; insertUnlocked(entry, min_unprocessed_insert_time_changed, lock); @@ -308,7 +309,7 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z LOG_DEBUG(log, "Pulling " << (end - begin) << " entries to queue: " << *begin << " - " << *last); - std::vector> futures; + std::vector>> futures; futures.reserve(end - begin); for (auto it = begin; it != end; ++it) @@ -324,11 +325,12 @@ bool ReplicatedMergeTreeQueue::pullLogsToQueue(zkutil::ZooKeeperPtr zookeeper, z for (auto & future : futures) { - zkutil::ZooKeeper::ValueAndStat res = future.second.get(); - copied_entries.emplace_back(LogEntry::parse(res.value, res.stat)); + zkutil::GetResponse res = future.second.get(); + + copied_entries.emplace_back(LogEntry::parse(res.data, res.stat)); ops.emplace_back(zkutil::makeCreateRequest( - replica_path + "/queue/queue-", res.value, zkutil::CreateMode::PersistentSequential)); + replica_path + "/queue/queue-", res.data, zkutil::CreateMode::PersistentSequential)); const auto & entry = *copied_entries.back(); if (entry.type == LogEntry::GET_PART) diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index a3b4c9f2fee..8fa8d7a9100 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -312,7 +312,7 @@ void ReplicatedMergeTreeRestartingThread::activateReplica() * This is possible only when session in ZooKeeper expires. */ String data; - Stat stat; + zkutil::Stat stat; bool has_is_active = zookeeper->tryGet(is_active_path, data, &stat); if (has_is_active && data == active_node_identifier) { diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp index 67660ea3446..e92856f213f 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp @@ -131,7 +131,7 @@ BlockInputStreams StorageSystemZooKeeper::read( if (path_part == "/") path_part.clear(); - std::vector futures; + std::vector> futures; futures.reserve(nodes.size()); for (const String & node : nodes) futures.push_back(zookeeper->asyncTryGet(path_part + '/' + node)); @@ -141,7 +141,7 @@ BlockInputStreams StorageSystemZooKeeper::read( for (size_t i = 0, size = nodes.size(); i < size; ++i) { auto res = futures[i].get(); - if (!res.exists) + if (res.error == ZooKeeperImpl::ZooKeeper::ZNONODE) continue; /// Node was deleted meanwhile. const zkutil::Stat & stat = res.stat; From 7ec66830afc8dc71e8f5d29d967980644c77355f Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Mar 2018 23:17:27 +0300 Subject: [PATCH 39/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Storages/StorageDistributed.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index cb3e1f3061a..0521a8d244f 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -194,8 +194,6 @@ BlockInputStreams StorageDistributed::read( if (settings.distributed_group_by_no_merge) processed_stage = QueryProcessingStage::Complete; - else if (settings.distributed_group_by_force_mergeable_state) - processed_stage = QueryProcessingStage::WithMergeableState; else /// Normal mode. processed_stage = result_size == 1 ? QueryProcessingStage::Complete From 87b4d8209a14278a3c27ad4cc3197ead45c5c527 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 24 Mar 2018 23:28:43 +0300 Subject: [PATCH 40/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- .../ZooKeeper/tests/zkutil_expiration_test.cpp | 14 +++++++------- .../Common/ZooKeeper/tests/zkutil_test_async.cpp | 4 ++-- .../src/Storages/System/StorageSystemZooKeeper.cpp | 2 +- 3 files changed, 10 insertions(+), 10 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp index 2c98847213c..28bd45dfe33 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_expiration_test.cpp @@ -34,17 +34,17 @@ int main(int argc, char ** argv) { { zkutil::Requests ops; - ops.emplace_back(zkutil::makeCreateRequest>("/test/zk_expiration_test", "hello", zkutil::CreateMode::Persistent)); - ops.emplace_back(zkutil::makeRemoveRequest>("/test/zk_expiration_test", -1)); + ops.emplace_back(zkutil::makeCreateRequest("/test/zk_expiration_test", "hello", zkutil::CreateMode::Persistent)); + ops.emplace_back(zkutil::makeRemoveRequest("/test/zk_expiration_test", -1)); - zkutil::MultiTransactionInfo info; - zk.tryMultiNoThrow(ops, nullptr, &info); + zkutil::Responses responses; + int32_t code = zk.tryMultiNoThrow(ops, responses); - std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(info.code) << std::endl; + std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(code) << std::endl; try { - if (info.code) - std::cout << "Path: " << info.getFailedOp().getPath() << std::endl; + if (code) + std::cout << "Path: " << zkutil::KeeperMultiException(code, ops, responses).getPathForFirstFailedOp() << std::endl; } catch (...) { diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp index 8bf47ecf05c..c9698bf6c11 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp @@ -25,10 +25,10 @@ try { std::vector> futures; for (auto & node : nodes) - futures.push_back(zookeeper.asyncTryGet("/tmp/" + node)); + futures.push_back(zookeeper.asyncGet("/tmp/" + node)); for (auto & future : futures) - std::cerr << (future.get().value.empty() ? ',' : '.'); + std::cerr << (future.get().data.empty() ? ',' : '.'); } }); } diff --git a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp index e92856f213f..e506802ec74 100644 --- a/dbms/src/Storages/System/StorageSystemZooKeeper.cpp +++ b/dbms/src/Storages/System/StorageSystemZooKeeper.cpp @@ -148,7 +148,7 @@ BlockInputStreams StorageSystemZooKeeper::read( size_t col_num = 0; res_columns[col_num++]->insert(nodes[i]); - res_columns[col_num++]->insert(res.value); + res_columns[col_num++]->insert(res.data); res_columns[col_num++]->insert(Int64(stat.czxid)); res_columns[col_num++]->insert(Int64(stat.mzxid)); res_columns[col_num++]->insert(UInt64(stat.ctime / 1000)); From 280a5f8e3f0e32a4303e85a5ce2e7c170111e191 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 02:01:21 +0300 Subject: [PATCH 41/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 6 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 181 +++++++++++------- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 19 +- .../ZooKeeper/tests/zkutil_test_async.cpp | 2 +- .../tests/zkutil_test_commands_new_lib.cpp | 2 +- .../Common/ZooKeeper/tests/zookeeper_impl.cpp | 2 +- 6 files changed, 138 insertions(+), 74 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index be80588bce6..cbb446d3de5 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -12,6 +12,7 @@ #include #define ZOOKEEPER_CONNECTION_TIMEOUT_MS 1000 +#define ZOOKEEPER_OPERATION_TIMEOUT_MS 1000 namespace ProfileEvents @@ -80,7 +81,8 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_, identity_.empty() ? "" : "digest", identity_, Poco::Timespan(0, session_timeout_ms_ * 1000), - Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000)); + Poco::Timespan(0, ZOOKEEPER_CONNECTION_TIMEOUT_MS * 1000), + Poco::Timespan(0, ZOOKEEPER_OPERATION_TIMEOUT_MS * 1000)); ProfileEvents::increment(ProfileEvents::ZooKeeperInit); @@ -166,6 +168,8 @@ ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std static WatchCallback callbackForEvent(const EventPtr & watch) { + if (!watch) + return {}; return [watch](const ZooKeeperImpl::ZooKeeper::WatchResponse &) { watch->set(); }; } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 16f4ee8608c..2a5f0eed5c4 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -9,7 +9,6 @@ #include #include -#include #include #include @@ -419,21 +418,13 @@ ZooKeeper::~ZooKeeper() { try { - /// Send close event. This also signals sending thread to wakeup and then stop. - if (!expired) - close(); + finalize(false, false); if (send_thread.joinable()) send_thread.join(); - /// This will also wakeup receiving thread. - socket.shutdown(); - if (receive_thread.joinable()) receive_thread.join(); - - /// Fire all remaining callbacks and watches. - finalize(); } catch (...) { @@ -448,9 +439,11 @@ ZooKeeper::ZooKeeper( const String & auth_scheme, const String & auth_data, Poco::Timespan session_timeout, - Poco::Timespan connection_timeout) + Poco::Timespan connection_timeout, + Poco::Timespan operation_timeout) : root_path(root_path_), - session_timeout(session_timeout) + session_timeout(session_timeout), + operation_timeout(operation_timeout) { if (!root_path.empty()) { @@ -521,8 +514,8 @@ void ZooKeeper::connect( if (!connected) throw Exception("All connection tries failed"); /// TODO more info; error code - socket.setReceiveTimeout(session_timeout); - socket.setSendTimeout(session_timeout); + socket.setReceiveTimeout(operation_timeout); + socket.setSendTimeout(operation_timeout); socket.setNoDelay(true); in.emplace(socket); @@ -608,29 +601,35 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data) void ZooKeeper::sendThread() { - auto prev_heartbeat_time = std::chrono::steady_clock::now(); + auto prev_heartbeat_time = clock::now(); try { while (!expired) { - auto now = std::chrono::steady_clock::now(); + auto now = clock::now(); auto next_heartbeat_time = prev_heartbeat_time + std::chrono::milliseconds(session_timeout.totalMilliseconds() / 3); - UInt64 max_wait = 0; + if (next_heartbeat_time > now) - max_wait = std::chrono::duration_cast(next_heartbeat_time - now).count(); - - RequestPtr request; - if (requests.tryPop(request, max_wait)) { - request->write(*out); + /// Wait for the next request in queue. No more than operation timeout. No more than until next heartbeat time. + UInt64 max_wait = std::min( + std::chrono::duration_cast(next_heartbeat_time - now).count(), + operation_timeout.totalMilliseconds()); - if (request->xid == close_xid) - break; + RequestPtr request; + if (requests.tryPop(request, max_wait)) + { + request->write(*out); + + if (request->xid == close_xid) + break; + } } else { - prev_heartbeat_time = std::chrono::steady_clock::now(); + /// Send heartbeat. + prev_heartbeat_time = clock::now(); HeartbeatRequest request; request.xid = ping_xid; @@ -641,10 +640,9 @@ void ZooKeeper::sendThread() catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); + finalize(true, false); } - expired = true; - /// Drain queue RequestPtr request; while (requests.tryPop(request)) @@ -656,23 +654,50 @@ void ZooKeeper::receiveThread() { try { + Int64 waited = 0; while (!expired) { - if (!in->poll(session_timeout.totalMicroseconds())) - throw Exception("Nothing is received in session timeout"); + clock::time_point now = clock::now(); + UInt64 max_wait = operation_timeout.totalMicroseconds(); + bool has_operations = false; - if (expired) - break; + { + std::lock_guard lock(operations_mutex); + if (!operations.empty()) + { + /// Operations are ordered by xid (and consequently, by time). + has_operations = true; + auto earliest_operation_deadline = operations.begin()->second.time + std::chrono::microseconds(operation_timeout.totalMicroseconds()); + if (now > earliest_operation_deadline) + throw Exception("Operation timeout"); + max_wait = std::chrono::duration_cast(earliest_operation_deadline - now).count(); + } + } - receiveEvent(); + if (in->poll(max_wait)) + { + if (expired) + break; + + receiveEvent(); + waited = 0; + } + else + { + if (has_operations) + throw Exception("Operation timeout"); + waited += max_wait; + if (waited > session_timeout.totalMicroseconds()) + throw Exception("Nothing is received in session timeout"); + + } } } catch (...) { tryLogCurrentException(__PRETTY_FUNCTION__); + finalize(false, true); } - - expired = true; } @@ -802,7 +827,7 @@ void ZooKeeper::receiveEvent() { for (auto & callback : it->second) if (callback) - callback(watch_response); + callback(watch_response); /// NOTE We may process callbacks not under mutex. watches.erase(it); } @@ -845,42 +870,67 @@ void ZooKeeper::receiveEvent() } -void ZooKeeper::finalize() +void ZooKeeper::finalize(bool error_send, bool error_receive) { + bool expired_prev = false; + if (expired.compare_exchange_strong(expired_prev, true)) { - std::lock_guard lock(operations_mutex); - - for (auto & op : operations) + try { - RequestInfo & request_info = op.second; - ResponsePtr response = request_info.request->makeResponse(); - response->error = ZCONNECTIONLOSS; - if (request_info.callback) - request_info.callback(*response); + if (!error_send) + { + /// Send close event. This also signals sending thread to wakeup and then stop. + close(); + send_thread.join(); + } + + /// This will also wakeup receiving thread. + socket.shutdown(); + + if (!error_receive) + receive_thread.join(); + + { + std::lock_guard lock(operations_mutex); + + for (auto & op : operations) + { + RequestInfo & request_info = op.second; + ResponsePtr response = request_info.request->makeResponse(); + response->error = ZCONNECTIONLOSS; + if (request_info.callback) + request_info.callback(*response); + } + + operations.clear(); + } + + { + std::lock_guard lock(watches_mutex); + + for (auto & path_watches : watches) + { + WatchResponse response; + response.type = SESSION; + response.state = EXPIRED_SESSION; + response.error = ZCONNECTIONLOSS; + + for (auto & callback : path_watches.second) + if (callback) + callback(response); + } + + watches.clear(); + } } - - operations.clear(); - } - - { - std::lock_guard lock(watches_mutex); - - for (auto & path_watches : watches) + catch (...) { - WatchResponse response; - response.type = SESSION; - response.state = EXPIRED_SESSION; - response.error = ZCONNECTIONLOSS; - - for (auto & callback : path_watches.second) - if (callback) - callback(response); + tryLogCurrentException(__PRETTY_FUNCTION__); } - - watches.clear(); } } + void ZooKeeper::AuthRequest::writeImpl(WriteBuffer & out) const { ZooKeeperImpl::write(type, out); @@ -1081,11 +1131,14 @@ void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in) void ZooKeeper::pushRequest(RequestInfo && info) { - if (expired) + /// If the request is close request, we push it even after session is expired - because it will signal sending thread to stop. + if (expired && info.request->xid != close_xid) throw Exception("Session expired"); info.request->addRootPath(root_path); + info.time = clock::now(); + if (!info.request->xid) { info.request->xid = xid.fetch_add(1); @@ -1259,8 +1312,6 @@ void ZooKeeper::multi( void ZooKeeper::close() { - /// TODO closed flag or make method private - CloseRequest request; request.xid = close_xid; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index be8fa89c1a5..01d6ca3c0a1 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -13,6 +13,7 @@ #include #include +#include #include #include #include @@ -331,14 +332,18 @@ public: }; - /// Connection to addresses is performed in order. If you want, shuffle them manually. + /** Connection to addresses is performed in order. If you want, shuffle them manually. + * Operation timeout couldn't be greater than session timeout. + * Operation timeout applies independently for network read, network write, waiting for events and synchronization. + */ ZooKeeper( const Addresses & addresses, const String & root_path, const String & auth_scheme, const String & auth_data, Poco::Timespan session_timeout, - Poco::Timespan connection_timeout); + Poco::Timespan connection_timeout, + Poco::Timespan operation_timeout); ~ZooKeeper(); @@ -403,8 +408,6 @@ public: const Requests & requests, MultiCallback callback); - void close(); - enum Error { @@ -476,6 +479,7 @@ private: ACLs default_acls; Poco::Timespan session_timeout; + Poco::Timespan operation_timeout; Poco::Net::StreamSocket socket; std::optional in; @@ -484,11 +488,14 @@ private: int64_t session_id = 0; std::atomic xid {1}; + using clock = std::chrono::steady_clock; + struct RequestInfo { RequestPtr request; ResponseCallback callback; WatchCallback watch; + clock::time_point time; }; using RequestsQueue = ConcurrentBoundedQueue; @@ -526,8 +533,10 @@ private: void sendThread(); void receiveThread(); + void close(); + /// Call all remaining callbacks and watches, passing errors to them. - void finalize(); + void finalize(bool error_send, bool error_receive); template void write(const T &); diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp index c9698bf6c11..fcd6ffd449d 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_async.cpp @@ -41,5 +41,5 @@ try catch (const Poco::Exception & e) { std::cout << e.message() << std::endl; - throw; + return 1; } diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp index 6dc9e3cb07b..71e52c6d277 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands_new_lib.cpp @@ -31,7 +31,7 @@ try for (const auto & address_string : addresses_strings) addresses.emplace_back(address_string); - ZooKeeper zk(addresses, {}, {}, {}, {5, 0}, {0, 50000}); + ZooKeeper zk(addresses, {}, {}, {}, {5, 0}, {0, 50000}, {0, 50000}); Poco::Event event(true); diff --git a/dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp b/dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp index 669d30a3d4b..ee775f21a44 100644 --- a/dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zookeeper_impl.cpp @@ -5,7 +5,7 @@ int main() try { - ZooKeeperImpl::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}); + ZooKeeperImpl::ZooKeeper zookeeper({Poco::Net::SocketAddress{"localhost:2181"}}, "", "", "", {30, 0}, {0, 50000}, {0, 50000}); zookeeper.create("/test", "hello", false, false, {}, [](const ZooKeeperImpl::ZooKeeper::CreateResponse & response) { From 0714a217fe53439d04c66d66ac423be0ea37ae50 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 02:13:05 +0300 Subject: [PATCH 42/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 10 +++++----- .../Common/ZooKeeper/tests/zkutil_test_commands.cpp | 4 +++- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 2a5f0eed5c4..2e21804026b 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -11,7 +11,7 @@ #include -#include +//#include /** ZooKeeper wire protocol. @@ -799,7 +799,7 @@ void ZooKeeper::receiveEvent() response = std::make_shared(); - std::cerr << "Received heartbeat\n"; +// std::cerr << "Received heartbeat\n"; } else if (xid == watch_xid) { @@ -833,7 +833,7 @@ void ZooKeeper::receiveEvent() } }; - std::cerr << "Received watch\n"; +// std::cerr << "Received watch\n"; } else { @@ -848,7 +848,7 @@ void ZooKeeper::receiveEvent() operations.erase(it); } - std::cerr << "Received response: " << request_info.request->getOpNum() << "\n"; +// std::cerr << "Received response: " << request_info.request->getOpNum() << "\n"; response = request_info.request->makeResponse(); } @@ -1083,7 +1083,7 @@ void ZooKeeper::MultiResponse::readImpl(ReadBuffer & in) ZooKeeperImpl::read(done, in); ZooKeeperImpl::read(op_error, in); - std::cerr << "Received result for multi: " << op_num << "\n"; +// std::cerr << "Received result for multi: " << op_num << "\n"; if (done) throw Exception("Not enough results received for multi transaction"); diff --git a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp index c8752a98f5f..670a7bcb75f 100644 --- a/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp +++ b/dbms/src/Common/ZooKeeper/tests/zkutil_test_commands.cpp @@ -1,7 +1,9 @@ #include +#include #include #include + using namespace zkutil; int main(int argc, char ** argv) @@ -41,7 +43,7 @@ try ops.emplace_back(zkutil::makeRemoveRequest("/test", -1)); std::cout << "multi" << std::endl; zkutil::Responses res = zk.multi(ops); - std::cout << "path created: " << dynamic_cast(*ops[0]).path_created << std::endl; + std::cout << "path created: " << typeid_cast(*res[0]).path_created << std::endl; return 0; } From 522bf01db9ad0164d6b883853f4933579ee947e4 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 03:15:52 +0300 Subject: [PATCH 43/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 14 ++--- dbms/src/Common/ZooKeeper/ZooKeeper.h | 2 +- dbms/src/Interpreters/DDLWorker.cpp | 3 +- dbms/src/Server/ClusterCopier.cpp | 57 +++++++++---------- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 3 +- .../ReplicatedMergeTreeRestartingThread.cpp | 3 +- .../Storages/StorageReplicatedMergeTree.cpp | 34 ++++------- 7 files changed, 50 insertions(+), 66 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index cbb446d3de5..11c7d5e8309 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -498,17 +498,10 @@ Responses ZooKeeper::multi(const Requests & requests) return responses; } -int32_t ZooKeeper::tryMulti(const Requests & requests) +int32_t ZooKeeper::tryMulti(const Requests & requests, Responses & responses) { - Responses responses; int32_t code = multiImpl(requests, responses); - - if (!(code == ZooKeeperImpl::ZooKeeper::ZOK || - code == ZooKeeperImpl::ZooKeeper::ZNONODE || - code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS || - code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS || - code == ZooKeeperImpl::ZooKeeper::ZBADVERSION || - code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY)) + if (code && !isUserError(code)) throw KeeperException(code); return code; } @@ -558,7 +551,8 @@ void ZooKeeper::tryRemoveChildrenRecursive(const std::string & path) /// Try to remove the children with a faster method - in bulk. If this fails, /// this means someone is concurrently removing these children and we will have /// to remove them one by one. - if (tryMulti(ops) != ZooKeeperImpl::ZooKeeper::ZOK) + Responses responses; + if (tryMulti(ops, responses) != ZooKeeperImpl::ZooKeeper::ZOK) for (const std::string & child : batch) tryRemove(child); } diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.h b/dbms/src/Common/ZooKeeper/ZooKeeper.h index 3d005eb6e0a..ede8c255a77 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.h @@ -148,7 +148,7 @@ public: Responses multi(const Requests & requests); /// Throws only if some operation has returned an "unexpected" error /// - an error that would cause the corresponding try- method to throw. - int32_t tryMulti(const Requests & requests); + int32_t tryMulti(const Requests & requests, Responses & responses); /// Throws nothing, just alias of multiImpl int32_t tryMultiNoThrow(const Requests & requests, Responses & responses) { diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 36eeae90272..ea176fb998b 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -808,7 +808,8 @@ void DDLWorker::createStatusDirs(const std::string & node_path) request.path = node_path + "/finished"; ops.emplace_back(std::make_shared(std::move(request))); } - int code = zookeeper->tryMulti(ops); + zkutil::Responses responses; + int code = zookeeper->tryMulti(ops, responses); if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw zkutil::KeeperException(code); } diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index 7f330d5de1f..9689aedf896 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -354,32 +354,26 @@ struct TaskCluster }; +struct MultiTransactionInfo +{ + int32_t code; + zkutil::Requests requests; + zkutil::Responses responses; +}; + + /// Atomically checks that is_dirty node is not exists, and made the remaining op /// Returns relative number of failed operation in the second field (the passed op has 0 index) -static zkutil::MultiTransactionInfo checkNoNodeAndCommit( +static MultiTransactionInfo checkNoNodeAndCommit( const zkutil::ZooKeeperPtr & zookeeper, const String & checking_node_path, zkutil::RequestPtr && op) { - zkutil::Requests ops; - { - zkutil::CreateRequest request; - request.path = checking_node_path; - ops.emplace_back(std::make_shared(std::move(request))); - } - { - zkutil::RemoveRequest request; - request.path = checking_node_path; - ops.emplace_back(std::make_shared(std::move(request))); - } - ops.emplace_back(std::move(op)); - - zkutil::Responses responses; - auto code = zookeeper->tryMultiNoThrow(ops, responses); - - if (code && !zkutil::isUserError(code)) - throw zkutil::KeeperException(code); - + MultiTransactionInfo info; + info.requests.emplace_back(zkutil::makeCreateRequest(checking_node_path, "", zkutil::CreateMode::Persistent)); + info.requests.emplace_back(zkutil::makeRemoveRequest(checking_node_path, -1)); + info.requests.emplace_back(std::move(op)); + info.code = zookeeper->tryMulti(info.requests, info.responses); return info; } @@ -740,7 +734,7 @@ public: { auto zookeeper = getZooKeeper(); - task_description_watch_callback = [this] (zkutil::ZooKeeper &, int, int, const char *) + task_description_watch_callback = [this] (const ZooKeeperImpl::ZooKeeper::WatchResponse &) { UInt64 version = ++task_descprtion_version; LOG_DEBUG(log, "Task description should be updated, local version " << version); @@ -1061,7 +1055,8 @@ protected: zkutil::Requests ops; ops.emplace_back(zkutil::makeSetRequest(workers_version_path, description, version)); ops.emplace_back(zkutil::makeCreateRequest(current_worker_path, description, zkutil::CreateMode::Ephemeral)); - auto code = zookeeper->tryMulti(ops); + zkutil::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); if (code == ZooKeeperImpl::ZooKeeper::ZOK || code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) return std::make_shared(current_worker_path, *zookeeper, false, false, description); @@ -1466,7 +1461,7 @@ protected: auto create_is_dirty_node = [&] () { auto code = zookeeper->tryCreate(is_dirty_flag_path, current_task_status_path, zkutil::CreateMode::Persistent); - if (code && code != ZNODEEXISTS) + if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw zkutil::KeeperException(code, is_dirty_flag_path); }; @@ -1587,17 +1582,19 @@ protected: { String start_state = TaskStateWithOwner::getData(TaskState::Started, host_id); auto op_create = zkutil::makeCreateRequest(current_task_status_path, start_state, zkutil::CreateMode::Persistent); - zkutil::MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create)); + MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_create)); if (info.code) { - if (info.getFailedOp().getPath() == is_dirty_flag_path) + zkutil::KeeperMultiException exception(info.code, info.requests, info.responses); + + if (exception.getPathForFirstFailedOp() == is_dirty_flag_path) { LOG_INFO(log, "Partition " << task_partition.name << " is dirty and will be dropped and refilled"); return PartitionTaskStatus::Error; } - throw zkutil::KeeperException(info.code, current_task_status_path); + throw exception; } } @@ -1674,7 +1671,7 @@ protected: if (future_is_dirty_checker != nullptr) { - zkutil::ZooKeeper::StatAndExists status; + zkutil::ExistsResponse status; try { status = future_is_dirty_checker->get(); @@ -1729,11 +1726,13 @@ protected: { String state_finished = TaskStateWithOwner::getData(TaskState::Finished, host_id); auto op_set = zkutil::makeSetRequest(current_task_status_path, state_finished, 0); - zkutil::MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set)); + MultiTransactionInfo info = checkNoNodeAndCommit(zookeeper, is_dirty_flag_path, std::move(op_set)); if (info.code) { - if (info.getFailedOp().getPath() == is_dirty_flag_path) + zkutil::KeeperMultiException exception(info.code, info.requests, info.responses); + + if (exception.getPathForFirstFailedOp() == is_dirty_flag_path) LOG_INFO(log, "Partition " << task_partition.name << " became dirty and will be dropped and refilled"); else LOG_INFO(log, "Someone made the node abandoned. Will refill partition. " << zkutil::ZooKeeper::error2string(info.code)); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp index 9360b2c62e6..cd4f587a1e5 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeQueue.cpp @@ -174,7 +174,8 @@ void ReplicatedMergeTreeQueue::updateTimesInZooKeeper( if (!ops.empty()) { - auto code = zookeeper->tryMulti(ops); + zkutil::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); if (code) LOG_ERROR(log, "Couldn't set value of nodes for insert times (" diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index 8fa8d7a9100..97fcb1523a8 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -261,8 +261,9 @@ void ReplicatedMergeTreeRestartingThread::removeFailedQuorumParts() LOG_DEBUG(log, "Found part " << part_name << " with failed quorum. Moving to detached. This shouldn't happen often."); zkutil::Requests ops; + zkutil::Responses responses; storage.removePartFromZooKeeper(part_name, ops); - auto code = zookeeper->tryMulti(ops); + auto code = zookeeper->tryMulti(ops, responses); if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) LOG_WARNING(log, "Part " << part_name << " with failed quorum is not in ZooKeeper. This shouldn't happen often."); diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 8a5f20b8421..7f96a177573 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -499,7 +499,8 @@ void StorageReplicatedMergeTree::createTableIfNotExists() ops.emplace_back(zkutil::makeCreateRequest(zookeeper_path + "/replicas", "", zkutil::CreateMode::Persistent)); - auto code = zookeeper->tryMulti(ops); + zkutil::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); if (code && code != ZooKeeperImpl::ZooKeeper::ZNODEEXISTS) throw zkutil::KeeperException(code); } @@ -1393,7 +1394,8 @@ bool StorageReplicatedMergeTree::executeFetch(const StorageReplicatedMergeTree:: if (!entry.block_id.empty() && zookeeper->exists(zookeeper_path + "/blocks/" + entry.block_id)) ops.emplace_back(zkutil::makeRemoveRequest(zookeeper_path + "/blocks/" + entry.block_id, -1)); - auto code = zookeeper->tryMulti(ops); + zkutil::Responses responses; + auto code = zookeeper->tryMulti(ops, responses); if (code == ZooKeeperImpl::ZooKeeper::ZOK) { @@ -1507,8 +1509,9 @@ void StorageReplicatedMergeTree::executeDropRange(const StorageReplicatedMergeTr data.renameAndDetachPart(part); zkutil::Requests ops; + zkutil::Responses responses; removePartFromZooKeeper(part->name, ops); - auto code = getZooKeeper()->tryMulti(ops); + auto code = getZooKeeper()->tryMulti(ops, responses); /// If the part is already removed (for example, because it was never added to ZK due to crash, /// see ReplicatedMergeTreeBlockOutputStream), then Ok. @@ -2170,9 +2173,10 @@ void StorageReplicatedMergeTree::updateQuorum(const String & part_name) /// The quorum is reached. Delete the node, and update information about the last part that was successfully written with quorum. zkutil::Requests ops; + zkutil::Responses responses; ops.emplace_back(zkutil::makeRemoveRequest(quorum_status_path, stat.version)); ops.emplace_back(zkutil::makeSetRequest(quorum_last_part_path, part_name, -1)); - auto code = zookeeper->tryMulti(ops); + auto code = zookeeper->tryMulti(ops, responses); if (code == ZooKeeperImpl::ZooKeeper::ZOK) { @@ -3632,22 +3636,6 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() } -static int32_t tryMulti(zkutil::ZooKeeperPtr & zookeeper, zkutil::Requests & ops) noexcept -{ - int32_t code; - try - { - code = zookeeper->tryMulti(ops); - } - catch (const zkutil::KeeperException & e) - { - code = e.code; - } - - return code; -} - - void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & zookeeper, const Strings & part_names, NameSet * parts_should_be_retied) { @@ -3661,8 +3649,8 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & auto it_next = std::next(it); if (ops.size() >= zkutil::MULTI_BATCH_SIZE || it_next == part_names.cend()) { - /// It is Ok to use multi with retries to delete nodes, because new nodes with the same names cannot appear here - auto code = tryMulti(zookeeper, ops); + zkutil::Responses unused_responses; + auto code = zookeeper->tryMultiNoThrow(ops, unused_responses); ops.clear(); if (code == ZooKeeperImpl::ZooKeeper::ZNONODE) @@ -3674,7 +3662,7 @@ void StorageReplicatedMergeTree::removePartsFromZooKeeper(zkutil::ZooKeeperPtr & { zkutil::Requests cur_ops; removePartFromZooKeeper(*it_in_batch, cur_ops); - auto cur_code = tryMulti(zookeeper, cur_ops); + auto cur_code = zookeeper->tryMultiNoThrow(cur_ops, unused_responses); if (cur_code == ZooKeeperImpl::ZooKeeper::ZNONODE) { From 548c4b0adf32cbe4e4d0db8ebe6cea16bc5b0b84 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 03:20:51 +0300 Subject: [PATCH 44/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 01d6ca3c0a1..512e6a9a421 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -349,7 +349,7 @@ public: /// If expired, you can only destroy the object. All other methods will throw exception. - bool isExpired() const { return !expired; } + bool isExpired() const { return expired; } /// Useful to check owner of ephemeral node. int64_t getSessionID() const { return session_id; } From 6721914041b31a5abc96a9f459d5e7fdaa3db6ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 03:36:35 +0300 Subject: [PATCH 45/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/KeeperException.h | 1 + dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 10 +++++----- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/KeeperException.h b/dbms/src/Common/ZooKeeper/KeeperException.h index 79dce371a19..0c40e9469a0 100644 --- a/dbms/src/Common/ZooKeeper/KeeperException.h +++ b/dbms/src/Common/ZooKeeper/KeeperException.h @@ -88,6 +88,7 @@ public: Responses responses; size_t failed_op_index = 0; + size_t getFailedOpIndex() const; std::string getPathForFirstFailedOp() const; /// If it is user error throws KeeperMultiException else throws ordinary KeeperException diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 11c7d5e8309..28e75c98049 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -794,7 +794,7 @@ std::future ZooKeeper::asyncMulti(const } -size_t getFailedOpIndex(const Responses & responses, int32_t transaction_return_code) +size_t KeeperMultiException::getFailedOpIndex() const { if (responses.empty()) throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR); @@ -803,8 +803,8 @@ size_t getFailedOpIndex(const Responses & responses, int32_t transaction_return_ if (responses[index]->error) return index; - if (!isUserError(transaction_return_code)) - throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(transaction_return_code) + "' is not valid response code for that", + if (!isUserError(code)) + throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(code) + "' is not valid response code for that", DB::ErrorCodes::LOGICAL_ERROR); throw DB::Exception("There is no failed OpResult", DB::ErrorCodes::LOGICAL_ERROR); @@ -812,8 +812,8 @@ size_t getFailedOpIndex(const Responses & responses, int32_t transaction_return_ KeeperMultiException::KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses) - : KeeperException("Transaction failed at op #" + std::to_string(getFailedOpIndex(responses, code)), code), - requests(requests), responses(responses), failed_op_index(getFailedOpIndex(responses, code)) + : KeeperException("Transaction failed at op #" + std::to_string(getFailedOpIndex()), code), + requests(requests), responses(responses) { } From 49045754d99a7b8ca339a5b52550c79484695711 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 03:56:08 +0300 Subject: [PATCH 46/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/KeeperException.h | 4 +++- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 17 ++++++++--------- 2 files changed, 11 insertions(+), 10 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/KeeperException.h b/dbms/src/Common/ZooKeeper/KeeperException.h index 0c40e9469a0..d310cb8dc1d 100644 --- a/dbms/src/Common/ZooKeeper/KeeperException.h +++ b/dbms/src/Common/ZooKeeper/KeeperException.h @@ -88,7 +88,6 @@ public: Responses responses; size_t failed_op_index = 0; - size_t getFailedOpIndex() const; std::string getPathForFirstFailedOp() const; /// If it is user error throws KeeperMultiException else throws ordinary KeeperException @@ -96,6 +95,9 @@ public: static void check(int32_t code, const Requests & requests, const Responses & responses); KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses); + +private: + size_t getFailedOpIndex(int32_t code, const Responses & responses) const; }; }; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 28e75c98049..d66cb4cb90f 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -477,8 +477,10 @@ int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses) auto callback = [&](const ZooKeeperImpl::ZooKeeper::MultiResponse & response) { code = response.error; - if (!code) - responses = response.responses; + responses = response.responses; + + std::cerr << code << ", " << responses.size() << "\n"; + event.set(); }; @@ -494,6 +496,7 @@ Responses ZooKeeper::multi(const Requests & requests) { Responses responses; int32_t code = multiImpl(requests, responses); + std::cerr << responses.size() << "\n"; KeeperMultiException::check(code, requests, responses); return responses; } @@ -516,11 +519,7 @@ void ZooKeeper::removeChildrenRecursive(const std::string & path) for (size_t i = 0; i < MULTI_BATCH_SIZE && !children.empty(); ++i) { removeChildrenRecursive(path + "/" + children.back()); - - ZooKeeperImpl::ZooKeeper::RemoveRequest request; - request.path = path + "/" + children.back(); - - ops.emplace_back(std::make_shared(std::move(request))); + ops.emplace_back(makeRemoveRequest(path + "/" + children.back(), -1)); children.pop_back(); } multi(ops); @@ -794,7 +793,7 @@ std::future ZooKeeper::asyncMulti(const } -size_t KeeperMultiException::getFailedOpIndex() const +size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & responses) const { if (responses.empty()) throw DB::Exception("Responses for multi transaction is empty", DB::ErrorCodes::LOGICAL_ERROR); @@ -812,7 +811,7 @@ size_t KeeperMultiException::getFailedOpIndex() const KeeperMultiException::KeeperMultiException(int32_t code, const Requests & requests, const Responses & responses) - : KeeperException("Transaction failed at op #" + std::to_string(getFailedOpIndex()), code), + : KeeperException("Transaction failed at op #" + std::to_string(getFailedOpIndex(code, responses)), code), requests(requests), responses(responses) { } From 2c4b28597e184c963dec5bc1aa033d32fae6279d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 03:57:04 +0300 Subject: [PATCH 47/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index d66cb4cb90f..9ad0f1a6e89 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -478,9 +478,6 @@ int32_t ZooKeeper::multiImpl(const Requests & requests, Responses & responses) { code = response.error; responses = response.responses; - - std::cerr << code << ", " << responses.size() << "\n"; - event.set(); }; @@ -496,7 +493,6 @@ Responses ZooKeeper::multi(const Requests & requests) { Responses responses; int32_t code = multiImpl(requests, responses); - std::cerr << responses.size() << "\n"; KeeperMultiException::check(code, requests, responses); return responses; } From c2f40e8d41bcb8ecdc5c4d6248da58a91114ed89 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 06:47:56 +0300 Subject: [PATCH 48/70] Removed libzookeeper [#CLICKHOUSE-2] --- .gitmodules | 3 -- CMakeLists.txt | 1 - cmake/find_zookeeper.cmake | 35 ------------------------ contrib/CMakeLists.txt | 4 --- contrib/zookeeper | 1 - dbms/src/Common/ZooKeeper/CMakeLists.txt | 7 +---- 6 files changed, 1 insertion(+), 50 deletions(-) delete mode 100644 cmake/find_zookeeper.cmake delete mode 160000 contrib/zookeeper diff --git a/.gitmodules b/.gitmodules index ab5c0c7bb74..a1ba915b91e 100644 --- a/.gitmodules +++ b/.gitmodules @@ -1,6 +1,3 @@ -[submodule "contrib/zookeeper"] - path = contrib/zookeeper - url = https://github.com/ClickHouse-Extras/zookeeper.git [submodule "contrib/poco"] path = contrib/poco url = https://github.com/ClickHouse-Extras/poco diff --git a/CMakeLists.txt b/CMakeLists.txt index ca4be1f5893..3c2586cffd0 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -258,7 +258,6 @@ include (cmake/find_sparsehash.cmake) include (cmake/find_rt.cmake) include (cmake/find_execinfo.cmake) include (cmake/find_readline_edit.cmake) -include (cmake/find_zookeeper.cmake) include (cmake/find_re2.cmake) include (cmake/find_rdkafka.cmake) include (cmake/find_capnp.cmake) diff --git a/cmake/find_zookeeper.cmake b/cmake/find_zookeeper.cmake deleted file mode 100644 index f667c310eb4..00000000000 --- a/cmake/find_zookeeper.cmake +++ /dev/null @@ -1,35 +0,0 @@ -if (ARCH_ARM) - # bundled have some asm broken for arm, use package libzookeeper-mt-dev - set(USE_INTERNAL_ZOOKEEPER_LIBRARY 0 CACHE BOOL "") -endif () - -option (USE_INTERNAL_ZOOKEEPER_LIBRARY "Set to FALSE to use system zookeeper library instead of bundled" ${NOT_UNBUNDLED}) - -if (USE_INTERNAL_ZOOKEEPER_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/zookeeper/src/c/CMakeLists.txt") - message (WARNING "submodule contrib/zookeeper is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_ZOOKEEPER_LIBRARY 0) -endif () - -if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY) - find_library (ZOOKEEPER_LIBRARY zookeeper_mt) - find_path (ZOOKEEPER_INCLUDE_DIR NAMES zookeeper/zookeeper.h PATHS ${ZOOKEEPER_INCLUDE_PATHS}) - set(ZOOKEEPER_INCLUDE_DIR "${ZOOKEEPER_INCLUDE_DIR}/zookeeper") -endif () - -if (ZOOKEEPER_LIBRARY AND ZOOKEEPER_INCLUDE_DIR) -else () - set (USE_INTERNAL_ZOOKEEPER_LIBRARY 1) - set(WANT_CPPUNIT 0 CACHE BOOL "") - set (ZOOKEEPER_LIBRARY zookeeper) -endif () - -message (STATUS "Using zookeeper: ${ZOOKEEPER_INCLUDE_DIR} : ${ZOOKEEPER_LIBRARY}") - - -# how to make clickhouse branch of https://github.com/ClickHouse-Extras/zookeeper.git : -# git remote add upstream https://github.com/apache/zookeeper.git -# git checkhout upstream/master -# git branch -D clickhouse -# git checkout -b clickhouse -# git merge clickhouse_misc -# git merge clickhouse_706 diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 9015f7e7d36..118c7009044 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -25,10 +25,6 @@ if (USE_INTERNAL_DOUBLE_CONVERSION_LIBRARY) add_subdirectory (double-conversion) endif () -if (USE_INTERNAL_ZOOKEEPER_LIBRARY) - add_subdirectory (zookeeper/src/c) -endif () - if (USE_INTERNAL_CITYHASH_LIBRARY) add_subdirectory (libcityhash) endif () diff --git a/contrib/zookeeper b/contrib/zookeeper deleted file mode 160000 index 438afae5af3..00000000000 --- a/contrib/zookeeper +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 438afae5af36c5be9c82d074f43a9bb19e0797c0 diff --git a/dbms/src/Common/ZooKeeper/CMakeLists.txt b/dbms/src/Common/ZooKeeper/CMakeLists.txt index 6a02d5733b1..197b53da00d 100644 --- a/dbms/src/Common/ZooKeeper/CMakeLists.txt +++ b/dbms/src/Common/ZooKeeper/CMakeLists.txt @@ -1,15 +1,10 @@ - include(${ClickHouse_SOURCE_DIR}/cmake/dbms_glob_sources.cmake) add_headers_and_sources(clickhouse_common_zookeeper .) add_library(clickhouse_common_zookeeper ${SPLIT_SHARED} ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources}) -if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY) - target_include_directories (clickhouse_common_zookeeper BEFORE PUBLIC ${ZOOKEEPER_INCLUDE_DIR}) -endif () - -target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io ${ZOOKEEPER_LIBRARY}) +target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io) if (ENABLE_TESTS) add_subdirectory (tests) From 93307a82e634068c03ed5281f4636d3c6198f6dd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 07:20:31 +0300 Subject: [PATCH 49/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 24 ++-- .../main.cpp | 1 + utils/zookeeper-dump-tree/main.cpp | 115 ++++-------------- utils/zookeeper-remove-by-list/main.cpp | 5 +- 4 files changed, 43 insertions(+), 102 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index 9ad0f1a6e89..0280992a8b6 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -634,10 +634,10 @@ std::future ZooKeeper::asyncGet(const std auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable + auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable { if (response.error) - promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); else promise->set_value(response); }; @@ -655,10 +655,10 @@ std::future ZooKeeper::asyncTryGet(const auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable + auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::GetResponse & response) mutable { if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); else promise->set_value(response); }; @@ -675,10 +675,10 @@ std::future ZooKeeper::asyncExists(con auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable + auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::ExistsResponse & response) mutable { if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE) - promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); else promise->set_value(response); }; @@ -696,10 +696,10 @@ std::future ZooKeeper::asyncGetChildren( auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable + auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::ListResponse & response) mutable { if (response.error) - promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); else promise->set_value(response); }; @@ -716,10 +716,10 @@ std::future ZooKeeper::asyncRemove(con auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable + auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable { if (response.error) - promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); else promise->set_value(response); }; @@ -736,10 +736,10 @@ std::future ZooKeeper::asyncTryRemove( auto promise = std::make_shared>(); auto future = promise->get_future(); - auto callback = [promise](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable + auto callback = [promise, path](const ZooKeeperImpl::ZooKeeper::RemoveResponse & response) mutable { if (response.error && response.error != ZooKeeperImpl::ZooKeeper::ZNONODE && response.error != ZooKeeperImpl::ZooKeeper::ZBADVERSION && response.error != ZooKeeperImpl::ZooKeeper::ZNOTEMPTY) - promise->set_exception(std::make_exception_ptr(KeeperException(response.error))); + promise->set_exception(std::make_exception_ptr(KeeperException(path, response.error))); else promise->set_value(response); }; diff --git a/utils/zookeeper-create-entry-to-download-part/main.cpp b/utils/zookeeper-create-entry-to-download-part/main.cpp index 050672956f7..5419c044c42 100644 --- a/utils/zookeeper-create-entry-to-download-part/main.cpp +++ b/utils/zookeeper-create-entry-to-download-part/main.cpp @@ -1,3 +1,4 @@ +#include #include #include #include diff --git a/utils/zookeeper-dump-tree/main.cpp b/utils/zookeeper-dump-tree/main.cpp index cabfe1c2068..1d614216a24 100644 --- a/utils/zookeeper-dump-tree/main.cpp +++ b/utils/zookeeper-dump-tree/main.cpp @@ -1,88 +1,10 @@ +#include #include #include #include -#include -#include +#include -/** Outputs paths of all ZK nodes in arbitrary order. Possibly only in specified directory. - */ - -struct CallbackState -{ - std::string path; - std::list::const_iterator it; - std::list::const_iterator> children; - Int64 dataLength = 0; -}; - -using CallbackStates = std::list; -CallbackStates states; - -zkutil::ZooKeeper * zookeeper; - -int running_count = 0; -Poco::Event completed; - - -void process(CallbackState & state); - -void callback( - int rc, - const String_vector * strings, - const Stat * stat, - const void * data) -{ - CallbackState * state = reinterpret_cast(const_cast(data)); - - if (rc != ZOK && rc != ZNONODE) - { - std::cerr << zerror(rc) << ", path: " << state->path << "\n"; - } - - if (stat != nullptr) - state->dataLength = stat->dataLength; - - if (rc == ZOK && strings) - { - for (int32_t i = 0; i < strings->count; ++i) - { - states.emplace_back(); - states.back().path = state->path + (state->path == "/" ? "" : "/") + strings->data[i]; - states.back().it = --states.end(); - state->children.push_back(states.back().it); - - process(states.back()); - } - } - - --running_count; - if (running_count == 0) - completed.set(); -} - -void process(CallbackState & state) -{ - ++running_count; - zoo_awget_children2(zookeeper->getHandle(), state.path.data(), nullptr, nullptr, callback, &state); -} - -typedef std::pair NodesBytes; - -NodesBytes printTree(const CallbackState & state) -{ - Int64 nodes = 1; - Int64 bytes = state.dataLength; - for (auto child : state.children) - { - NodesBytes nodesBytes = printTree(*child); - nodes += nodesBytes.first; - bytes += nodesBytes.second; - } - std::cout << state.path << '\t' << nodes << '\t' << bytes <<'\n'; - return NodesBytes(nodes, bytes); -} - int main(int argc, char ** argv) try { @@ -106,18 +28,35 @@ try return 1; } - ZooKeeperImpl::ZooKeeper zookeeper(options.at("address").as()); - zookeeper = &zookeeper_; + zkutil::ZooKeeper zookeeper(options.at("address").as()); - states.emplace_back(); - states.back().path = options.at("path").as(); - states.back().it = --states.end(); + std::string initial_path = options.at("path").as(); - process(states.back()); + std::list>> list_futures; + list_futures.emplace_back(initial_path, zookeeper.asyncGetChildren(initial_path)); - completed.wait(); + for (auto it = list_futures.begin(); it != list_futures.end(); ++it) + { + zkutil::ListResponse response; + try + { + response = it->second.get(); + } + catch (const zkutil::KeeperException & e) + { + if (e.code == ZooKeeperImpl::ZooKeeper::ZNONODE) + continue; + throw; + } - printTree(*states.begin()); + std::cout << it->first << '\t' << response.stat.numChildren << '\t' << response.stat.dataLength << '\n'; + + for (const auto & name : response.names) + { + std::string child_path = it->first == "/" ? it->first + name : it->first + '/' + name; + list_futures.emplace_back(child_path, zookeeper.asyncGetChildren(child_path)); + } + } } catch (...) { diff --git a/utils/zookeeper-remove-by-list/main.cpp b/utils/zookeeper-remove-by-list/main.cpp index 74e46c42e16..8ac90f4712e 100644 --- a/utils/zookeeper-remove-by-list/main.cpp +++ b/utils/zookeeper-remove-by-list/main.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -28,7 +29,7 @@ try zkutil::ZooKeeper zookeeper(options.at("address").as()); DB::ReadBufferFromFileDescriptor in(STDIN_FILENO); - std::list futures; + std::list> futures; std::cerr << "Requested: "; while (!in.eof()) @@ -36,7 +37,7 @@ try std::string path; DB::readEscapedString(path, in); DB::assertString("\n", in); - futures.push_back(zookeeper.asyncRemove(path)); + futures.emplace_back(zookeeper.asyncRemove(path)); std::cerr << "."; } std::cerr << "\n"; From 5ac329c91cde85d15312d8f5f87f1271564a0312 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 25 Mar 2018 07:34:33 +0300 Subject: [PATCH 50/70] Using different ZooKeeper library (development) [#CLICKHOUSE-2] --- .../gtest_zkutil_test_multi_exception.cpp | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 deletions(-) diff --git a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp index f4f0d0ac86d..1a564854ebf 100644 --- a/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp +++ b/dbms/src/Common/ZooKeeper/tests/gtest_zkutil_test_multi_exception.cpp @@ -1,3 +1,4 @@ +#include #include #include #include @@ -86,14 +87,13 @@ TEST(zkutil, multi_async) ops.clear(); auto res = fut.get(); - ASSERT_TRUE(res.code == ZooKeeperImpl::ZooKeeper::ZOK); - ASSERT_EQ(res.results->size(), 2); - ASSERT_EQ(res.ops_ptr->size(), 2); + ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZOK); + ASSERT_EQ(res.responses.size(), 2); } EXPECT_ANY_THROW ( - std::vector> futures; + std::vector> futures; for (size_t i = 0; i < 10000; ++i) { @@ -123,9 +123,8 @@ TEST(zkutil, multi_async) ops.clear(); auto res = fut.get(); - ASSERT_TRUE(res.code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS); - ASSERT_EQ(res.results->size(), 2); - ASSERT_EQ(res.ops_ptr->size(), 2); + ASSERT_TRUE(res.error == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS); + ASSERT_EQ(res.responses.size(), 2); } } @@ -144,7 +143,7 @@ TEST(zkutil, multi_async_libzookeeper_segfault) auto future = zookeeper->asyncMulti(ops); auto res = future.get(); - EXPECT_TRUE(zkutil::isHardwareError(res.code)); + EXPECT_TRUE(zkutil::isHardwareError(res.error)); } @@ -165,12 +164,12 @@ TEST(zkutil, multi_create_sequential) String sequential_node_prefix = base_path + "/queue-"; ops.emplace_back(zkutil::makeCreateRequest(sequential_node_prefix, "", zkutil::CreateMode::EphemeralSequential)); - zkutil::OpResultsPtr results = zookeeper->multi(ops); - zkutil::OpResult & sequential_node_result_op = results->at(0); + auto results = zookeeper->multi(ops); + const auto & sequential_node_result_op = typeid_cast(*results.at(0)); - EXPECT_FALSE(sequential_node_result_op.value.empty()); - EXPECT_GT(sequential_node_result_op.value.length(), sequential_node_prefix.length()); - EXPECT_EQ(sequential_node_result_op.value.substr(0, sequential_node_prefix.length()), sequential_node_prefix); + EXPECT_FALSE(sequential_node_result_op.path_created.empty()); + EXPECT_GT(sequential_node_result_op.path_created.length(), sequential_node_prefix.length()); + EXPECT_EQ(sequential_node_result_op.path_created.substr(0, sequential_node_prefix.length()), sequential_node_prefix); } catch (...) { From 17ee06b0d93919e3eb07d240eed383f7d8d0af6a Mon Sep 17 00:00:00 2001 From: BayoNet Date: Sun, 25 Mar 2018 05:04:22 +0300 Subject: [PATCH 51/70] English translation update. --- docs/en/agg_functions/combinators.md | 0 docs/en/agg_functions/index.md | 1 + docs/en/agg_functions/parametric_functions.md | 0 docs/en/agg_functions/reference.md | 34 ++--- docs/en/data_types/array.md | 0 docs/en/data_types/boolean.md | 0 docs/en/data_types/date.md | 2 +- docs/en/data_types/datetime.md | 0 docs/en/data_types/enum.md | 0 docs/en/data_types/fixedstring.md | 0 docs/en/data_types/float.md | 10 +- docs/en/data_types/index.md | 5 +- docs/en/data_types/int_uint.md | 0 .../aggregatefunction.md | 0 .../nested_data_structures/index.md | 0 .../nested_data_structures/nested.md | 0 .../special_data_types/expression.md | 0 .../en/data_types/special_data_types/index.md | 0 docs/en/data_types/special_data_types/set.md | 0 docs/en/data_types/string.md | 0 docs/en/data_types/tuple.md | 0 docs/en/development/style.md | 65 +++++----- docs/en/dicts/external_dicts.md | 5 +- docs/en/dicts/external_dicts_dict.md | 9 +- docs/en/dicts/external_dicts_dict_layout.md | 99 +++------------ docs/en/dicts/external_dicts_dict_lifetime.md | 6 +- docs/en/dicts/external_dicts_dict_sources.md | 6 +- .../en/dicts/external_dicts_dict_structure.md | 14 +- docs/en/dicts/index.md | 0 docs/en/dicts/internal_dicts.md | 0 docs/en/formats/capnproto.md | 52 ++++---- docs/en/formats/csv.md | 0 docs/en/formats/csvwithnames.md | 0 docs/en/formats/index.md | 1 + docs/en/formats/json.md | 0 docs/en/formats/jsoncompact.md | 2 +- docs/en/formats/jsoneachrow.md | 0 docs/en/formats/native.md | 0 docs/en/formats/null.md | 0 docs/en/formats/pretty.md | 0 docs/en/formats/prettycompact.md | 0 docs/en/formats/prettycompactmonoblock.md | 0 docs/en/formats/prettynoescapes.md | 0 docs/en/formats/prettyspace.md | 0 docs/en/formats/rowbinary.md | 2 +- docs/en/formats/tabseparated.md | 0 docs/en/formats/tabseparatedraw.md | 0 docs/en/formats/tabseparatedwithnames.md | 0 .../formats/tabseparatedwithnamesandtypes.md | 0 docs/en/formats/tskv.md | 0 docs/en/formats/values.md | 2 +- docs/en/formats/vertical.md | 0 docs/en/formats/xml.md | 2 +- docs/en/functions/arithmetic_functions.md | 0 docs/en/functions/array_functions.md | 12 +- docs/en/functions/array_join.md | 1 - docs/en/functions/bit_functions.md | 1 - docs/en/functions/comparison_functions.md | 2 +- docs/en/functions/conditional_functions.md | 0 docs/en/functions/date_time_functions.md | 4 + docs/en/functions/encoding_functions.md | 0 docs/en/functions/ext_dict_functions.md | 8 +- docs/en/functions/hash_functions.md | 0 docs/en/functions/higher_order_functions.md | 3 +- docs/en/functions/in_functions.md | 0 docs/en/functions/index.md | 0 docs/en/functions/ip_address_functions.md | 0 docs/en/functions/json_functions.md | 9 +- docs/en/functions/logical_functions.md | 1 - docs/en/functions/math_functions.md | 1 - docs/en/functions/other_functions.md | 5 +- docs/en/functions/random_functions.md | 0 docs/en/functions/rounding_functions.md | 0 .../functions/splitting_merging_functions.md | 0 docs/en/functions/string_functions.md | 0 docs/en/functions/string_replace_functions.md | 1 - docs/en/functions/string_search_functions.md | 0 .../en/functions/type_conversion_functions.md | 0 docs/en/functions/url_functions.md | 0 docs/en/functions/ym_dict_functions.md | 12 +- .../example_datasets/amplab_benchmark.md | 0 .../example_datasets/criteo.md | 4 +- .../example_datasets/nyc_taxi.md | 15 +-- .../example_datasets/ontime.md | 0 .../example_datasets/star_schema.md | 3 +- .../example_datasets/wikistat.md | 2 +- docs/en/getting_started/index.md | 14 +- docs/en/index.md | 2 +- docs/en/interfaces/cli.md | 7 +- docs/en/interfaces/http_interface.md | 16 +-- docs/en/interfaces/index.md | 3 +- docs/en/interfaces/jdbc.md | 0 docs/en/interfaces/tcp.md | 0 .../third-party_client_libraries.md | 2 +- docs/en/interfaces/third-party_gui.md | 0 docs/en/introduction/distinctive_features.md | 20 +-- .../features_considered_disadvantages.md | 0 docs/en/introduction/index.md | 1 + docs/en/introduction/performance.md | 0 .../introduction/possible_silly_questions.md | 4 +- docs/en/introduction/ya_metrika_task.md | 4 +- docs/en/operations/access_rights.md | 59 ++++----- docs/en/operations/configuration_files.md | 2 +- docs/en/operations/index.md | 3 +- docs/en/operations/quotas.md | 35 +++-- docs/en/operations/server_settings/index.md | 1 + .../en/operations/server_settings/settings.md | 37 +++--- docs/en/operations/settings/index.md | 4 +- .../operations/settings/query_complexity.md | 0 docs/en/operations/settings/settings.md | 16 +-- .../operations/settings/settings_profiles.md | 2 +- docs/en/operations/tips.md | 5 +- docs/en/operators/index.md | 4 +- docs/en/query_language/index.md | 1 + docs/en/query_language/queries.md | 53 ++++---- docs/en/query_language/syntax.md | 2 +- docs/en/roadmap.md | 70 +++++----- docs/en/system_tables/index.md | 2 +- .../system.asynchronous_metrics.md | 0 docs/en/system_tables/system.clusters.md | 17 ++- docs/en/system_tables/system.columns.md | 1 - docs/en/system_tables/system.databases.md | 0 docs/en/system_tables/system.dictionaries.md | 26 ++-- docs/en/system_tables/system.events.md | 0 docs/en/system_tables/system.functions.md | 2 +- docs/en/system_tables/system.merges.md | 1 - docs/en/system_tables/system.numbers.md | 0 docs/en/system_tables/system.numbers_mt.md | 0 docs/en/system_tables/system.one.md | 0 docs/en/system_tables/system.parts.md | 0 docs/en/system_tables/system.processes.md | 6 +- docs/en/system_tables/system.replicas.md | 26 ++-- docs/en/system_tables/system.settings.md | 4 +- docs/en/system_tables/system.tables.md | 3 +- docs/en/system_tables/system.zookeeper.md | 1 - docs/en/table_engines/aggregatingmergetree.md | 31 +++-- docs/en/table_engines/buffer.md | 0 docs/en/table_engines/collapsingmergetree.md | 0 .../table_engines/custom_partitioning_key.md | 14 +- docs/en/table_engines/dictionary.md | 106 ++++++++++++++++ docs/en/table_engines/distributed.md | 46 +++---- docs/en/table_engines/external_data.md | 0 docs/en/table_engines/file.md | 0 docs/en/table_engines/graphitemergetree.md | 5 +- docs/en/table_engines/index.md | 0 docs/en/table_engines/join.md | 0 docs/en/table_engines/kafka.md | 2 +- docs/en/table_engines/log.md | 0 docs/en/table_engines/materializedview.md | 2 +- docs/en/table_engines/memory.md | 0 docs/en/table_engines/merge.md | 0 docs/en/table_engines/mergetree.md | 2 +- docs/en/table_engines/null.md | 0 docs/en/table_engines/replacingmergetree.md | 0 docs/en/table_engines/replication.md | 2 +- docs/en/table_engines/set.md | 0 docs/en/table_engines/summingmergetree.md | 0 docs/en/table_engines/tinylog.md | 0 docs/en/table_engines/view.md | 0 docs/en/table_functions/index.md | 0 docs/en/table_functions/merge.md | 0 docs/en/table_functions/remote.md | 2 +- docs/en/utils/clickhouse-copier.md | 120 +++++++++--------- docs/en/utils/clickhouse-local.md | 6 +- docs/en/utils/index.md | 7 +- docs/mkdocs_en.yml | 2 + docs/ru/dicts/external_dicts_dict_layout.md | 18 +-- 167 files changed, 648 insertions(+), 574 deletions(-) mode change 100644 => 100755 docs/en/agg_functions/combinators.md mode change 100644 => 100755 docs/en/agg_functions/index.md mode change 100644 => 100755 docs/en/agg_functions/parametric_functions.md mode change 100644 => 100755 docs/en/agg_functions/reference.md mode change 100644 => 100755 docs/en/data_types/array.md mode change 100644 => 100755 docs/en/data_types/boolean.md mode change 100644 => 100755 docs/en/data_types/date.md mode change 100644 => 100755 docs/en/data_types/datetime.md mode change 100644 => 100755 docs/en/data_types/enum.md mode change 100644 => 100755 docs/en/data_types/fixedstring.md mode change 100644 => 100755 docs/en/data_types/float.md mode change 100644 => 100755 docs/en/data_types/index.md mode change 100644 => 100755 docs/en/data_types/int_uint.md mode change 100644 => 100755 docs/en/data_types/nested_data_structures/aggregatefunction.md mode change 100644 => 100755 docs/en/data_types/nested_data_structures/index.md mode change 100644 => 100755 docs/en/data_types/nested_data_structures/nested.md mode change 100644 => 100755 docs/en/data_types/special_data_types/expression.md mode change 100644 => 100755 docs/en/data_types/special_data_types/index.md mode change 100644 => 100755 docs/en/data_types/special_data_types/set.md mode change 100644 => 100755 docs/en/data_types/string.md mode change 100644 => 100755 docs/en/data_types/tuple.md mode change 100644 => 100755 docs/en/development/style.md mode change 100644 => 100755 docs/en/dicts/external_dicts.md mode change 100644 => 100755 docs/en/dicts/external_dicts_dict.md mode change 100644 => 100755 docs/en/dicts/external_dicts_dict_layout.md mode change 100644 => 100755 docs/en/dicts/external_dicts_dict_lifetime.md mode change 100644 => 100755 docs/en/dicts/external_dicts_dict_sources.md mode change 100644 => 100755 docs/en/dicts/external_dicts_dict_structure.md mode change 100644 => 100755 docs/en/dicts/index.md mode change 100644 => 100755 docs/en/dicts/internal_dicts.md mode change 100644 => 100755 docs/en/formats/capnproto.md mode change 100644 => 100755 docs/en/formats/csv.md mode change 100644 => 100755 docs/en/formats/csvwithnames.md mode change 100644 => 100755 docs/en/formats/index.md mode change 100644 => 100755 docs/en/formats/json.md mode change 100644 => 100755 docs/en/formats/jsoncompact.md mode change 100644 => 100755 docs/en/formats/jsoneachrow.md mode change 100644 => 100755 docs/en/formats/native.md mode change 100644 => 100755 docs/en/formats/null.md mode change 100644 => 100755 docs/en/formats/pretty.md mode change 100644 => 100755 docs/en/formats/prettycompact.md mode change 100644 => 100755 docs/en/formats/prettycompactmonoblock.md mode change 100644 => 100755 docs/en/formats/prettynoescapes.md mode change 100644 => 100755 docs/en/formats/prettyspace.md mode change 100644 => 100755 docs/en/formats/rowbinary.md mode change 100644 => 100755 docs/en/formats/tabseparated.md mode change 100644 => 100755 docs/en/formats/tabseparatedraw.md mode change 100644 => 100755 docs/en/formats/tabseparatedwithnames.md mode change 100644 => 100755 docs/en/formats/tabseparatedwithnamesandtypes.md mode change 100644 => 100755 docs/en/formats/tskv.md mode change 100644 => 100755 docs/en/formats/values.md mode change 100644 => 100755 docs/en/formats/vertical.md mode change 100644 => 100755 docs/en/formats/xml.md mode change 100644 => 100755 docs/en/functions/arithmetic_functions.md mode change 100644 => 100755 docs/en/functions/array_functions.md mode change 100644 => 100755 docs/en/functions/array_join.md mode change 100644 => 100755 docs/en/functions/bit_functions.md mode change 100644 => 100755 docs/en/functions/comparison_functions.md mode change 100644 => 100755 docs/en/functions/conditional_functions.md mode change 100644 => 100755 docs/en/functions/date_time_functions.md mode change 100644 => 100755 docs/en/functions/encoding_functions.md mode change 100644 => 100755 docs/en/functions/ext_dict_functions.md mode change 100644 => 100755 docs/en/functions/hash_functions.md mode change 100644 => 100755 docs/en/functions/higher_order_functions.md mode change 100644 => 100755 docs/en/functions/in_functions.md mode change 100644 => 100755 docs/en/functions/index.md mode change 100644 => 100755 docs/en/functions/ip_address_functions.md mode change 100644 => 100755 docs/en/functions/json_functions.md mode change 100644 => 100755 docs/en/functions/logical_functions.md mode change 100644 => 100755 docs/en/functions/math_functions.md mode change 100644 => 100755 docs/en/functions/other_functions.md mode change 100644 => 100755 docs/en/functions/random_functions.md mode change 100644 => 100755 docs/en/functions/rounding_functions.md mode change 100644 => 100755 docs/en/functions/splitting_merging_functions.md mode change 100644 => 100755 docs/en/functions/string_functions.md mode change 100644 => 100755 docs/en/functions/string_replace_functions.md mode change 100644 => 100755 docs/en/functions/string_search_functions.md mode change 100644 => 100755 docs/en/functions/type_conversion_functions.md mode change 100644 => 100755 docs/en/functions/url_functions.md mode change 100644 => 100755 docs/en/functions/ym_dict_functions.md mode change 100644 => 100755 docs/en/getting_started/example_datasets/amplab_benchmark.md mode change 100644 => 100755 docs/en/getting_started/example_datasets/criteo.md mode change 100644 => 100755 docs/en/getting_started/example_datasets/nyc_taxi.md mode change 100644 => 100755 docs/en/getting_started/example_datasets/ontime.md mode change 100644 => 100755 docs/en/getting_started/example_datasets/star_schema.md mode change 100644 => 100755 docs/en/getting_started/example_datasets/wikistat.md mode change 100644 => 100755 docs/en/getting_started/index.md mode change 100644 => 100755 docs/en/index.md mode change 100644 => 100755 docs/en/interfaces/cli.md mode change 100644 => 100755 docs/en/interfaces/http_interface.md mode change 100644 => 100755 docs/en/interfaces/index.md mode change 100644 => 100755 docs/en/interfaces/jdbc.md mode change 100644 => 100755 docs/en/interfaces/tcp.md mode change 100644 => 100755 docs/en/interfaces/third-party_client_libraries.md mode change 100644 => 100755 docs/en/interfaces/third-party_gui.md mode change 100644 => 100755 docs/en/introduction/distinctive_features.md mode change 100644 => 100755 docs/en/introduction/features_considered_disadvantages.md mode change 100644 => 100755 docs/en/introduction/index.md mode change 100644 => 100755 docs/en/introduction/performance.md mode change 100644 => 100755 docs/en/introduction/possible_silly_questions.md mode change 100644 => 100755 docs/en/introduction/ya_metrika_task.md mode change 100644 => 100755 docs/en/operations/access_rights.md mode change 100644 => 100755 docs/en/operations/configuration_files.md mode change 100644 => 100755 docs/en/operations/index.md mode change 100644 => 100755 docs/en/operations/quotas.md mode change 100644 => 100755 docs/en/operations/server_settings/index.md mode change 100644 => 100755 docs/en/operations/server_settings/settings.md mode change 100644 => 100755 docs/en/operations/settings/index.md mode change 100644 => 100755 docs/en/operations/settings/query_complexity.md mode change 100644 => 100755 docs/en/operations/settings/settings.md mode change 100644 => 100755 docs/en/operations/settings/settings_profiles.md mode change 100644 => 100755 docs/en/operations/tips.md mode change 100644 => 100755 docs/en/operators/index.md mode change 100644 => 100755 docs/en/query_language/index.md mode change 100644 => 100755 docs/en/query_language/queries.md mode change 100644 => 100755 docs/en/query_language/syntax.md mode change 100644 => 100755 docs/en/roadmap.md mode change 100644 => 100755 docs/en/system_tables/index.md mode change 100644 => 100755 docs/en/system_tables/system.asynchronous_metrics.md mode change 100644 => 100755 docs/en/system_tables/system.clusters.md mode change 100644 => 100755 docs/en/system_tables/system.columns.md mode change 100644 => 100755 docs/en/system_tables/system.databases.md mode change 100644 => 100755 docs/en/system_tables/system.dictionaries.md mode change 100644 => 100755 docs/en/system_tables/system.events.md mode change 100644 => 100755 docs/en/system_tables/system.functions.md mode change 100644 => 100755 docs/en/system_tables/system.merges.md mode change 100644 => 100755 docs/en/system_tables/system.numbers.md mode change 100644 => 100755 docs/en/system_tables/system.numbers_mt.md mode change 100644 => 100755 docs/en/system_tables/system.one.md mode change 100644 => 100755 docs/en/system_tables/system.parts.md mode change 100644 => 100755 docs/en/system_tables/system.processes.md mode change 100644 => 100755 docs/en/system_tables/system.replicas.md mode change 100644 => 100755 docs/en/system_tables/system.settings.md mode change 100644 => 100755 docs/en/system_tables/system.tables.md mode change 100644 => 100755 docs/en/system_tables/system.zookeeper.md mode change 100644 => 100755 docs/en/table_engines/aggregatingmergetree.md mode change 100644 => 100755 docs/en/table_engines/buffer.md mode change 100644 => 100755 docs/en/table_engines/collapsingmergetree.md mode change 100644 => 100755 docs/en/table_engines/custom_partitioning_key.md create mode 100755 docs/en/table_engines/dictionary.md mode change 100644 => 100755 docs/en/table_engines/distributed.md mode change 100644 => 100755 docs/en/table_engines/external_data.md mode change 100644 => 100755 docs/en/table_engines/file.md mode change 100644 => 100755 docs/en/table_engines/graphitemergetree.md mode change 100644 => 100755 docs/en/table_engines/index.md mode change 100644 => 100755 docs/en/table_engines/join.md mode change 100644 => 100755 docs/en/table_engines/kafka.md mode change 100644 => 100755 docs/en/table_engines/log.md mode change 100644 => 100755 docs/en/table_engines/materializedview.md mode change 100644 => 100755 docs/en/table_engines/memory.md mode change 100644 => 100755 docs/en/table_engines/merge.md mode change 100644 => 100755 docs/en/table_engines/mergetree.md mode change 100644 => 100755 docs/en/table_engines/null.md mode change 100644 => 100755 docs/en/table_engines/replacingmergetree.md mode change 100644 => 100755 docs/en/table_engines/replication.md mode change 100644 => 100755 docs/en/table_engines/set.md mode change 100644 => 100755 docs/en/table_engines/summingmergetree.md mode change 100644 => 100755 docs/en/table_engines/tinylog.md mode change 100644 => 100755 docs/en/table_engines/view.md mode change 100644 => 100755 docs/en/table_functions/index.md mode change 100644 => 100755 docs/en/table_functions/merge.md mode change 100644 => 100755 docs/en/table_functions/remote.md mode change 100644 => 100755 docs/en/utils/clickhouse-copier.md mode change 100644 => 100755 docs/en/utils/clickhouse-local.md mode change 100644 => 100755 docs/en/utils/index.md diff --git a/docs/en/agg_functions/combinators.md b/docs/en/agg_functions/combinators.md old mode 100644 new mode 100755 diff --git a/docs/en/agg_functions/index.md b/docs/en/agg_functions/index.md old mode 100644 new mode 100755 index e87bf4ff833..3864f7271c4 --- a/docs/en/agg_functions/index.md +++ b/docs/en/agg_functions/index.md @@ -8,3 +8,4 @@ ClickHouse also supports: - [Parametric aggregate functions](parametric_functions.md#aggregate_functions_parametric), which accept other parameters in addition to columns. - [Combinators](combinators.md#aggregate_functions_combinators), which change the behavior of aggregate functions. + diff --git a/docs/en/agg_functions/parametric_functions.md b/docs/en/agg_functions/parametric_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/agg_functions/reference.md b/docs/en/agg_functions/reference.md old mode 100644 new mode 100755 index 90ff9da58e7..0eb896e4664 --- a/docs/en/agg_functions/reference.md +++ b/docs/en/agg_functions/reference.md @@ -19,7 +19,7 @@ In some cases, you can rely on the order of execution. This applies to cases whe When a `SELECT` query has the `GROUP BY` clause or at least one aggregate function, ClickHouse (in contrast to MySQL) requires that all expressions in the `SELECT`, `HAVING`, and `ORDER BY` clauses be calculated from keys or from aggregate functions. In other words, each column selected from the table must be used either in keys or inside aggregate functions. To get behavior like in MySQL, you can put the other columns in the `any` aggregate function. -## anyHeavy(x) +## anyHeavy Selects a frequently occurring value using the [heavy hitters](http://www.cs.umd.edu/~samir/498/karp.pdf) algorithm. If there is a value that occurs more than in half the cases in each of the query's execution threads, this value is returned. Normally, the result is nondeterministic. @@ -39,7 +39,6 @@ Take the [OnTime](../getting_started/example_datasets/ontime.md#example_datasets SELECT anyHeavy(AirlineID) AS res FROM ontime ``` - ``` ┌───res─┐ │ 19690 │ @@ -125,11 +124,11 @@ The result is always Float64. Calculates the approximate number of different values of the argument. Works for numbers, strings, dates, date-with-time, and for multiple arguments and tuple arguments. Uses an adaptive sampling algorithm: for the calculation state, it uses a sample of element hash values with a size up to 65536. -This algorithm is also very accurate for data sets with small cardinality (up to 65536) and very efficient on CPU (when computing not too many of these functions, using `uniq` is almost as fast as using other aggregate functions). +This algorithm is also very accurate for data sets with low cardinality (up to 65536) and very efficient on CPU (when computing not too many of these functions, using `uniq` is almost as fast as using other aggregate functions). The result is determinate (it doesn't depend on the order of query processing). -This function provides excellent accuracy even for data sets with huge cardinality (10B+ elements) and is recommended for use by default. +This function provides excellent accuracy even for data sets with extremely high cardinality (over 10 billion elements). It is recommended for default use. ## uniqCombined(x) @@ -139,16 +138,16 @@ A combination of three algorithms is used: array, hash table and [HyperLogLog](h The result is determinate (it doesn't depend on the order of query processing). -The `uniqCombined` function is a good default choice for calculating the number of different values, but the following should be considered: for data sets with large cardinality (200M+) error of estimate will only grow and for data sets with huge cardinality(1B+ elements) it returns result with high inaccuracy. +The `uniqCombined` function is a good default choice for calculating the number of different values, but keep in mind that the estimation error will increase for high-cardinality data sets (200M+ elements), and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements). ## uniqHLL12(x) Uses the [HyperLogLog](https://en.wikipedia.org/wiki/HyperLogLog) algorithm to approximate the number of different values of the argument. -212 5-bit cells are used. The size of the state is slightly more than 2.5 KB. Result is not very accurate (error up to ~10%) for data sets of small cardinality(<10K elements), but for data sets with large cardinality (10K - 100M) result is quite accurate (error up to ~1.6%) and after that error of estimate will only grow and for data sets with huge cardinality (1B+ elements) it returns result with high inaccuracy. +212 5-bit cells are used. The size of the state is slightly more than 2.5 KB. The result is not very accurate (up to ~10% error) for small data sets (<10K elements). However, the result is fairly accurate for high-cardinality data sets (10K-100M), with a maximum error of ~1.6%. Starting from 100M, the estimation error increases, and the function will return very inaccurate results for data sets with extremely high cardinality (1B+ elements). The result is determinate (it doesn't depend on the order of query processing). -This function is not recommended for use, and in most cases, use the `uniq` or `uniqCombined` function. +We don't recommend using this function. In most cases, use the `uniq` or `uniqCombined` function. ## uniqExact(x) @@ -170,7 +169,7 @@ In some cases, you can still rely on the order of execution. This applies to cas -## groupArrayInsertAt(x) +## groupArrayInsertAt Inserts a value into the array in the specified position. @@ -236,8 +235,8 @@ For its purpose (calculating quantiles of page loading times), using this functi ## quantileTimingWeighted(level)(x, weight) -Differs from the 'quantileTiming' function in that it has a second argument, "weights". Weight is a non-negative integer. -The result is calculated as if the 'x' value were passed 'weight' number of times to the 'quantileTiming' function. +Differs from the `quantileTiming` function in that it has a second argument, "weights". Weight is a non-negative integer. +The result is calculated as if the `x` value were passed `weight` number of times to the `quantileTiming` function. ## quantileExact(level)(x) @@ -257,7 +256,7 @@ The performance of the function is lower than for ` quantile`, ` quantileTiming` The result depends on the order of running the query, and is nondeterministic. -## median(x) +## median All the quantile functions have corresponding median functions: `median`, `medianDeterministic`, `medianTiming`, `medianTimingWeighted`, `medianExact`, `medianExactWeighted`, `medianTDigest`. They are synonyms and their behavior is identical. @@ -275,7 +274,7 @@ Returns `Float64`. When `n <= 1`, returns `+∞`. ## varPop(x) -Calculates the amount `Σ((x - x̅)^2) / n`, where `n` is the sample size and `x̅`is the average value of `x`. +Calculates the amount `Σ((x - x̅)^2) / (n - 1)`, where `n` is the sample size and `x̅`is the average value of `x`. In other words, dispersion for a set of values. Returns `Float64`. @@ -287,30 +286,33 @@ The result is equal to the square root of `varSamp(x)`. The result is equal to the square root of `varPop(x)`. -## topK(N)(column) +## topK Returns an array of the most frequent values in the specified column. The resulting array is sorted in descending order of frequency of values (not by the values themselves). Implements the [ Filtered Space-Saving](http://www.l2f.inesc-id.pt/~fmmb/wiki/uploads/Work/misnis.ref0a.pdf) algorithm for analyzing TopK, based on the reduce-and-combine algorithm from [Parallel Space Saving](https://arxiv.org/pdf/1401.0702.pdf). +``` +topK(N)(column) +``` + This function doesn't provide a guaranteed result. In certain situations, errors might occur and it might return frequent values that aren't the most frequent values. We recommend using the `N < 10 ` value; performance is reduced with large `N` values. Maximum value of ` N = 65536`. **Arguments** -- 'N' – The number of values. +- 'N' is the number of values. - ' x ' – The column. **Example** -Take the [OnTime](../getting_started/example_datasets/ontime.md#example_datasets-ontime)data set and select the three most frequently occurring values in the `AirlineID` column. +Take the [OnTime](../getting_started/example_datasets/ontime.md#example_datasets-ontime) data set and select the three most frequently occurring values in the `AirlineID` column. ```sql SELECT topK(3)(AirlineID) AS res FROM ontime ``` - ``` ┌─res─────────────────┐ │ [19393,19790,19805] │ diff --git a/docs/en/data_types/array.md b/docs/en/data_types/array.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/boolean.md b/docs/en/data_types/boolean.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/date.md b/docs/en/data_types/date.md old mode 100644 new mode 100755 index 355e5555bfa..cb179c0d8c4 --- a/docs/en/data_types/date.md +++ b/docs/en/data_types/date.md @@ -1,6 +1,6 @@ # Date -Date. Stored in two bytes as the number of days since 1970-01-01 (unsigned). Allows storing values from just after the beginning of the Unix Epoch to the upper threshold defined by a constant at the compilation stage (currently, this is until the year 2038, but it may be expanded to 2106). +A date. Stored in two bytes as the number of days since 1970-01-01 (unsigned). Allows storing values from just after the beginning of the Unix Epoch to the upper threshold defined by a constant at the compilation stage (currently, this is until the year 2106, but the final fully-supported year is 2105). The minimum value is output as 0000-00-00. The date is stored without the time zone. diff --git a/docs/en/data_types/datetime.md b/docs/en/data_types/datetime.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/enum.md b/docs/en/data_types/enum.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/fixedstring.md b/docs/en/data_types/fixedstring.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/float.md b/docs/en/data_types/float.md old mode 100644 new mode 100755 index c6d12999604..9d5cc2c01bb --- a/docs/en/data_types/float.md +++ b/docs/en/data_types/float.md @@ -4,8 +4,8 @@ Types are equivalent to types of C: -- `Float32` - `float`; -- `Float64` - ` double`. +- `Float32` - `float` +- `Float64` - ` double` We recommend that you store data in integer form whenever possible. For example, convert fixed precision numbers to integer values, such as monetary amounts or page load times in milliseconds. @@ -24,9 +24,7 @@ SELECT 1 - 0.9 ``` - The result of the calculation depends on the calculation method (the processor type and architecture of the computer system). - - Floating-point calculations might result in numbers such as infinity (`Inf`) and "not-a-number" (`NaN`). This should be taken into account when processing the results of calculations. - - When reading floating point numbers from rows, the result might not be the nearest machine-representable number. ## NaN and Inf @@ -44,6 +42,7 @@ SELECT 0.5 / 0 │ inf │ └────────────────┘ ``` + - `-Inf` – Negative infinity. ```sql @@ -55,6 +54,7 @@ SELECT -0.5 / 0 │ -inf │ └─────────────────┘ ``` + - `NaN` – Not a number. ``` @@ -67,5 +67,5 @@ SELECT 0 / 0 └──────────────┘ ``` - See the rules for ` NaN` sorting in the section [ORDER BY clause](../query_language/queries.md#query_language-queries-order_by). +See the rules for ` NaN` sorting in the section [ORDER BY clause](../query_language/queries.md#query_language-queries-order_by). diff --git a/docs/en/data_types/index.md b/docs/en/data_types/index.md old mode 100644 new mode 100755 index c17b51c08a2..4008a872161 --- a/docs/en/data_types/index.md +++ b/docs/en/data_types/index.md @@ -2,6 +2,7 @@ # Data types -ClickHouse table fields can contain data of different types. +ClickHouse can store various types of data in table cells. + +This section describes the supported data types and special considerations when using and/or implementing them, if any. -The topic contains descriptions of data types supported and specificity of their usage of implementation if exists. \ No newline at end of file diff --git a/docs/en/data_types/int_uint.md b/docs/en/data_types/int_uint.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/nested_data_structures/aggregatefunction.md b/docs/en/data_types/nested_data_structures/aggregatefunction.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/nested_data_structures/index.md b/docs/en/data_types/nested_data_structures/index.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/nested_data_structures/nested.md b/docs/en/data_types/nested_data_structures/nested.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/special_data_types/expression.md b/docs/en/data_types/special_data_types/expression.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/special_data_types/index.md b/docs/en/data_types/special_data_types/index.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/special_data_types/set.md b/docs/en/data_types/special_data_types/set.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/string.md b/docs/en/data_types/string.md old mode 100644 new mode 100755 diff --git a/docs/en/data_types/tuple.md b/docs/en/data_types/tuple.md old mode 100644 new mode 100755 diff --git a/docs/en/development/style.md b/docs/en/development/style.md old mode 100644 new mode 100755 index ef6490187c8..700fede5373 --- a/docs/en/development/style.md +++ b/docs/en/development/style.md @@ -4,7 +4,7 @@ 1. The following are recommendations, not requirements. 2. If you are editing code, it makes sense to follow the formatting of the existing code. -3. Code style is needed for consistency. Consistency makes it easier to read the code. and it also makes it easier to search the code. +3. Code style is needed for consistency. Consistency makes it easier to read the code, and it also makes it easier to search the code. 4. Many of the rules do not have logical reasons; they are dictated by established practices. ## Formatting @@ -93,25 +93,25 @@ 14. In classes and structures, public, private, and protected are written on the same level as the class/struct, but all other internal elements should be deeper. ```cpp - template > - class MultiVersion - { - public: - /// The specific version of the object to use. - using Version = Ptr; - ... - } + template +class MultiVersion +{ +public: + /// Version of object for usage. shared_ptr manage lifetime of version. + using Version = std::shared_ptr; + ... +} ``` 15. If the same namespace is used for the entire file, and there isn't anything else significant, an offset is not necessary inside namespace. 16. If the block for if, for, while... expressions consists of a single statement, you don't need to use curly brackets. Place the statement on a separate line, instead. The same is true for a nested if, for, while... statement. But if the inner statement contains curly brackets or else, the external block should be written in curly brackets. - ```cpp - /// Finish write. - for (auto & stream : streams) - stream.second->finalize(); - ``` + ```cpp + /// Finish write. +for (auto & stream : streams) + stream.second->finalize(); + ``` 17. There should be any spaces at the ends of lines. @@ -218,11 +218,11 @@ */ void executeQuery( ReadBuffer & istr, /// Where to read the query from (and data for INSERT, if applicable) - WriteBuffer & ostr, /// Where to write the result - Context & context, /// DB, tables, data types, engines, functions, aggregate functions... - BlockInputStreamPtr & query_plan, /// A description of query processing can be included here - QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// The last stage to process the SELECT query to - ) + WriteBuffer & ostr, /// Where to write the result + Context & context, /// DB, tables, data types, engines, functions, aggregate functions... + BlockInputStreamPtr & query_plan, /// A description of query processing can be included here + QueryProcessingStage::Enum stage = QueryProcessingStage::Complete /// The last stage to process the SELECT query to + ) ``` 4. Comments should be written in English only. @@ -252,7 +252,7 @@ */ ``` - (Example taken from: [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/)](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/) + (the example is borrowed from the resource [http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/](http://home.tamk.fi/~jaalto/course/coding-style/doc/unmaintainable-code/) 7. Do not write garbage comments (author, creation date ..) at the beginning of each file. @@ -497,7 +497,15 @@ This is not recommended, but it is allowed. You can create a separate code block inside a single function in order to make certain variables local, so that the destructors are called when exiting the block. ```cpp - Block block = data.in->read();{ std::lock_guard lock(mutex); data.ready = true; data.block = block;}ready_any.set(); + Block block = data.in->read(); + + { + std::lock_guard lock(mutex); + data.ready = true; + data.block = block; + } + + ready_any.set(); ``` 7. Multithreading. @@ -560,13 +568,12 @@ This is not recommended, but it is allowed. ```cpp using AggregateFunctionPtr = std::shared_ptr; - - /** Creates an aggregate function by name. - */ + + /** Creates an aggregate function by name. */ class AggregateFunctionFactory { public: - AggregateFunctionFactory(); + AggregateFunctionFactory(); AggregateFunctionPtr get(const String & name, const DataTypes & argument_types) const; ``` @@ -591,10 +598,10 @@ This is not recommended, but it is allowed. If later you’ll need to delay initialization, you can add a default constructor that will create an invalid object. Or, for a small number of objects, you can use shared_ptr/unique_ptr. ```cpp - Loader(DB::Connection * connection_, const std::string & query, size_t max_block_size_); - - /// For delayed initialization - Loader() {} + Loader(DB::Connection * connection_, const std::string & query, size_t max_block_size_); + + /// For delayed initialization + Loader() {} ``` 17. Virtual functions. diff --git a/docs/en/dicts/external_dicts.md b/docs/en/dicts/external_dicts.md old mode 100644 new mode 100755 index b99b02bbf57..a6af84a313f --- a/docs/en/dicts/external_dicts.md +++ b/docs/en/dicts/external_dicts.md @@ -21,12 +21,11 @@ The dictionary config file has the following format: /etc/metrika.xml - - + - + ... diff --git a/docs/en/dicts/external_dicts_dict.md b/docs/en/dicts/external_dicts_dict.md old mode 100644 new mode 100755 index 4133b036e1f..6d2f4128704 --- a/docs/en/dicts/external_dicts_dict.md +++ b/docs/en/dicts/external_dicts_dict.md @@ -27,7 +27,8 @@ The dictionary configuration has the following structure: ``` - name – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`. -- [source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources) – Source of the dictionary. -- [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) – Location of the dictionary in memory. -- [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) – Structure of the dictionary. A key and attributes that can be retrieved by this key. -- [lifetime](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) – How frequently to update dictionaries. +- [source](external_dicts_dict_sources.html/#dicts-external_dicts_dict_sources) — Source of the dictionary . +- [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) — Dictionary layout in memory. +- [source](external_dicts_dict_sources.html/#dicts-external_dicts_dict_sources) — Structure of the dictionary . A key and attributes that can be retrieved by this key. +- [lifetime](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) — Frequency of dictionary updates. + diff --git a/docs/en/dicts/external_dicts_dict_layout.md b/docs/en/dicts/external_dicts_dict_layout.md old mode 100644 new mode 100755 index ad635db94f5..8b7cad24b65 --- a/docs/en/dicts/external_dicts_dict_layout.md +++ b/docs/en/dicts/external_dicts_dict_layout.md @@ -2,11 +2,11 @@ # Storing dictionaries in memory -There are [many different ways](external_dicts_dict_layout#dicts-external_dicts_dict_layout-manner) to store dictionaries in memory. +There are a [variety of ways](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-manner) to store dictionaries in memory. -We recommend [flat](external_dicts_dict_layout#dicts-external_dicts_dict_layout-flat), [hashed](external_dicts_dict_layout#dicts-external_dicts_dict_layout-hashed), and [complex_key_hashed](external_dicts_dict_layout#dicts-external_dicts_dict_layout-complex_key_hashed). which provide optimal processing speed. +We recommend [flat](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-flat), [hashed](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-hashed)and[complex_key_hashed](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-complex_key_hashed). which provide optimal processing speed. -Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more about this in the "[cache](external_dicts_dict_layout#dicts-external_dicts_dict_layout-cache)" section. +Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more in the section " [cache](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-cache)". There are several ways to improve dictionary performance: @@ -46,7 +46,6 @@ The configuration looks like this: - [range_hashed](#dicts-external_dicts_dict_layout-range_hashed) - [complex_key_hashed](#dicts-external_dicts_dict_layout-complex_key_hashed) - [complex_key_cache](#dicts-external_dicts_dict_layout-complex_key_cache) -- [ip_trie](#dicts-external_dicts_dict_layout-ip_trie) @@ -88,7 +87,7 @@ Configuration example: ### complex_key_hashed -This type of storage is designed for use with compound [keys](external_dicts_dict_structure#dicts-external_dicts_dict_structure). It is similar to hashed. +This type is for use with composite [keys](external_dicts_dict_structure.md/#dicts-external_dicts_dict_structure). Similar to `hashed`. Configuration example: @@ -109,18 +108,18 @@ This storage method works the same way as hashed and allows using date/time rang Example: The table contains discounts for each advertiser in the format: ``` - +---------------+---------------------+-------------------+--------+ - | advertiser id | discount start date | discount end date | amount | - +===============+=====================+===================+========+ - | 123 | 2015-01-01 | 2015-01-15 | 0.15 | - +---------------+---------------------+-------------------+--------+ - | 123 | 2015-01-16 | 2015-01-31 | 0.25 | - +---------------+---------------------+-------------------+--------+ - | 456 | 2015-01-01 | 2015-01-15 | 0.05 | - +---------------+---------------------+-------------------+--------+ ++---------------+---------------------+-------------------+--------+ +| advertiser id | discount start date | discount end date | amount | ++===============+=====================+===================+========+ +| 123 | 2015-01-01 | 2015-01-15 | 0.15 | ++---------------+---------------------+-------------------+--------+ +| 123 | 2015-01-16 | 2015-01-31 | 0.25 | ++---------------+---------------------+-------------------+--------+ +| 456 | 2015-01-01 | 2015-01-15 | 0.05 | ++---------------+---------------------+-------------------+--------+ ``` -To use a sample for date ranges, define `range_min` and `range_max` in [structure](external_dicts_dict_structure#dicts-external_dicts_dict_structure). +To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure). Example: @@ -197,15 +196,15 @@ This is the least effective of all the ways to store dictionaries. The speed of To improve cache performance, use a subquery with ` LIMIT`, and call the function with the dictionary externally. -Supported [sources](external_dicts_dict_sources#dicts-external_dicts_dict_sources): MySQL, ClickHouse, executable, HTTP. +Supported [sources](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources): MySQL, ClickHouse, executable, HTTP. Example of settings: ```xml - - 1000000000 + + 1000000000 ``` @@ -227,66 +226,4 @@ Do not use ClickHouse as a source, because it is slow to process queries with ra ### complex_key_cache -This type of storage is designed for use with compound [keys](external_dicts_dict_structure#dicts-external_dicts_dict_structure). Similar to `cache`. - - - -### ip_trie - - -The table stores IP prefixes for each key (IP address), which makes it possible to map IP addresses to metadata such as ASN or threat score. - -Example: in the table there are prefixes matches to AS number and country: - -``` - +-----------------+-------+--------+ - | prefix | asn | cca2 | - +=================+=======+========+ - | 202.79.32.0/20 | 17501 | NP | - +-----------------+-------+--------+ - | 2620:0:870::/48 | 3856 | US | - +-----------------+-------+--------+ - | 2a02:6b8:1::/48 | 13238 | RU | - +-----------------+-------+--------+ - | 2001:db8::/32 | 65536 | ZZ | - +-----------------+-------+--------+ -``` - -When using such a layout, the structure should have the "key" element. - -Example: - -```xml - - - - prefix - String - - - - asn - UInt32 - - - - cca2 - String - ?? - - ... -``` - -These key must have only one attribute of type String, containing a valid IP prefix. Other types are not yet supported. - -For querying, same functions (dictGetT with tuple) as for complex key dictionaries have to be used: - - dictGetT('dict_name', 'attr_name', tuple(ip)) - -The function accepts either UInt32 for IPv4 address or FixedString(16) for IPv6 address in wire format: - - dictGetString('prefix', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) - -No other type is supported. The function returns attribute for a prefix matching the given IP address. If there are overlapping prefixes, the most specific one is returned. - -The data is stored currently in a bitwise trie, it has to fit in memory. +This type of storage is for use with composite [keys](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure). Similar to `cache`. diff --git a/docs/en/dicts/external_dicts_dict_lifetime.md b/docs/en/dicts/external_dicts_dict_lifetime.md old mode 100644 new mode 100755 index 6431fb3de48..52ee7a4aa78 --- a/docs/en/dicts/external_dicts_dict_lifetime.md +++ b/docs/en/dicts/external_dicts_dict_lifetime.md @@ -36,13 +36,13 @@ Example of settings: When upgrading the dictionaries, the ClickHouse server applies different logic depending on the type of [ source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources): > - For a text file, it checks the time of modification. If the time differs from the previously recorded time, the dictionary is updated. -> - For MyISAM tables, the time of modification is checked using a `SHOW TABLE STATUS` query. -> - Dictionaries from other sources are updated every time by default. +- For MyISAM tables, the time of modification is checked using a `SHOW TABLE STATUS` query. +- Dictionaries from other sources are updated every time by default. For MySQL (InnoDB) and ODBC sources, you can set up a query that will update the dictionaries only if they really changed, rather than each time. To do this, follow these steps: > - The dictionary table must have a field that always changes when the source data is updated. -> - The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. The query must be specified in the `` field in the [ source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources) settings. +- The settings of the source must specify a query that retrieves the changing field. The ClickHouse server interprets the query result as a row, and if this row has changed relative to its previous state, the dictionary is updated. Specify the query in the `` field in the settings for the [source](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources). Example of settings: diff --git a/docs/en/dicts/external_dicts_dict_sources.md b/docs/en/dicts/external_dicts_dict_sources.md old mode 100644 new mode 100755 index 721302cd556..6cb4e0ea44d --- a/docs/en/dicts/external_dicts_dict_sources.md +++ b/docs/en/dicts/external_dicts_dict_sources.md @@ -80,7 +80,7 @@ Setting fields: ## HTTP(s) -Working with executable files depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. +Working with an HTTP(s) server depends on [how the dictionary is stored in memory](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout). If the dictionary is stored using `cache` and `complex_key_cache`, ClickHouse requests the necessary keys by sending a request via the `POST` method. Example of settings: @@ -135,9 +135,9 @@ Installing unixODBC and the ODBC driver for PostgreSQL: Configuring `/etc/odbc.ini` (or `~/.odbc.ini`): ``` - [DEFAULT] +[DEFAULT] Driver = myconnection - + [myconnection] Description = PostgreSQL connection to my_db Driver = PostgreSQL Unicode diff --git a/docs/en/dicts/external_dicts_dict_structure.md b/docs/en/dicts/external_dicts_dict_structure.md old mode 100644 new mode 100755 index 5a6d349b350..2542af00ec6 --- a/docs/en/dicts/external_dicts_dict_structure.md +++ b/docs/en/dicts/external_dicts_dict_structure.md @@ -25,8 +25,8 @@ Overall structure: Columns are described in the structure: -- `` – [Key column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-key). -- `` – [Data column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-attributes). There can be a large number of columns. +- `` - [key column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-key). +- `` - [data column](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-attributes). There can be a large number of columns. @@ -63,12 +63,10 @@ Configuration fields: ### Composite key -The key can be a `tuple` from any types of fields. The [ layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) in this case must be `complex_key_hashed` or `complex_key_cache`. +The key can be a `tuple` from any types of fields. The [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) in this case must be `complex_key_hashed` or `complex_key_cache`.
- -A composite key can also consist of a single element, which makes it possible to use a string as the key, for instance. - +A composite key can consist of a single element. This makes it possible to use a string as the key, for instance.
The key structure is set in the element ``. Key fields are specified in the same format as the dictionary [attributes](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-attributes). Example: @@ -119,6 +117,6 @@ Configuration fields: - `null_value` – The default value for a non-existing element. In the example, it is an empty string. - `expression` – The attribute can be an expression. The tag is not required. - `hierarchical` – Hierarchical support. Mirrored to the parent identifier. By default, ` false`. -- `injective` Whether the `id -> attribute` image is injective. If ` true`, then you can optimize the ` GROUP BY` clause. By default, `false`. -- `is_object_id` - Used for query mongo documents by ObjectId +- `injective` – Whether the `id -> attribute` image is injective. If ` true`, then you can optimize the ` GROUP BY` clause. By default, `false`. +- `is_object_id` – Whether the query is executed for a MongoDB document by `ObjectID`. diff --git a/docs/en/dicts/index.md b/docs/en/dicts/index.md old mode 100644 new mode 100755 diff --git a/docs/en/dicts/internal_dicts.md b/docs/en/dicts/internal_dicts.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/capnproto.md b/docs/en/formats/capnproto.md old mode 100644 new mode 100755 index 0d482e20887..918197b2bd9 --- a/docs/en/formats/capnproto.md +++ b/docs/en/formats/capnproto.md @@ -1,26 +1,26 @@ - - -# CapnProto - -Cap'n Proto is a binary message format similar to Protocol Buffers and Thrift, but not like JSON or MessagePack. - -Cap'n Proto messages are strictly typed and not self-describing, meaning they need an external schema description. The schema is applied on the fly and cached for each query. - -```sql -SELECT SearchPhrase, count() AS c FROM test.hits - GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' -``` - -Where `schema.capnp` looks like this: - -``` -struct Message { - SearchPhrase @0 :Text; - c @1 :Uint64; -} -``` - -Schema files are in the file that is located in the directory specified in [ format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) in the server configuration. - -Deserialization is effective and usually doesn't increase the system load. - + + +# CapnProto + +Cap'n Proto is a binary message format similar to Protocol Buffers and Thrift, but not like JSON or MessagePack. + +Cap'n Proto messages are strictly typed and not self-describing, meaning they need an external schema description. The schema is applied on the fly and cached for each query. + +```sql +SELECT SearchPhrase, count() AS c FROM test.hits + GROUP BY SearchPhrase FORMAT CapnProto SETTINGS schema = 'schema:Message' +``` + +Where `schema.capnp` looks like this: + +``` +struct Message { + SearchPhrase @0 :Text; + c @1 :Uint64; +} +``` + +Schema files are in the file that is located in the directory specified in [ format_schema_path](../operations/server_settings/settings.md#server_settings-format_schema_path) in the server configuration. + +Deserialization is effective and usually doesn't increase the system load. + diff --git a/docs/en/formats/csv.md b/docs/en/formats/csv.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/csvwithnames.md b/docs/en/formats/csvwithnames.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/index.md b/docs/en/formats/index.md old mode 100644 new mode 100755 index 815a2d060cb..112a13ff5e5 --- a/docs/en/formats/index.md +++ b/docs/en/formats/index.md @@ -3,3 +3,4 @@ # Formats The format determines how data is returned to you after SELECTs (how it is written and formatted by the server), and how it is accepted for INSERTs (how it is read and parsed by the server). + diff --git a/docs/en/formats/json.md b/docs/en/formats/json.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/jsoncompact.md b/docs/en/formats/jsoncompact.md old mode 100644 new mode 100755 index e4ce0867bc2..d870b6dff08 --- a/docs/en/formats/jsoncompact.md +++ b/docs/en/formats/jsoncompact.md @@ -24,7 +24,7 @@ Example: ["bathroom interior design", "2166"], ["yandex", "1655"], ["spring 2014 fashion", "1549"], - ["freeform photos", "1480"] + ["freeform photo", "1480"] ], "totals": ["","8873898"], diff --git a/docs/en/formats/jsoneachrow.md b/docs/en/formats/jsoneachrow.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/native.md b/docs/en/formats/native.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/null.md b/docs/en/formats/null.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/pretty.md b/docs/en/formats/pretty.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/prettycompact.md b/docs/en/formats/prettycompact.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/prettycompactmonoblock.md b/docs/en/formats/prettycompactmonoblock.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/prettynoescapes.md b/docs/en/formats/prettynoescapes.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/prettyspace.md b/docs/en/formats/prettyspace.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/rowbinary.md b/docs/en/formats/rowbinary.md old mode 100644 new mode 100755 index bc8479332ba..aeb3df4c8a8 --- a/docs/en/formats/rowbinary.md +++ b/docs/en/formats/rowbinary.md @@ -9,5 +9,5 @@ Date is represented as a UInt16 object that contains the number of days since 19 String is represented as a varint length (unsigned [LEB128](https://en.wikipedia.org/wiki/LEB128)), followed by the bytes of the string. FixedString is represented simply as a sequence of bytes. -Arrays are represented as a varint length (unsigned [LEB128](https://en.wikipedia.org/wiki/LEB128)), followed by the array elements in order. +Array is represented as a varint length (unsigned [LEB128](https://en.wikipedia.org/wiki/LEB128)), followed by successive elements of the array. diff --git a/docs/en/formats/tabseparated.md b/docs/en/formats/tabseparated.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/tabseparatedraw.md b/docs/en/formats/tabseparatedraw.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/tabseparatedwithnames.md b/docs/en/formats/tabseparatedwithnames.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/tabseparatedwithnamesandtypes.md b/docs/en/formats/tabseparatedwithnamesandtypes.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/tskv.md b/docs/en/formats/tskv.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/values.md b/docs/en/formats/values.md old mode 100644 new mode 100755 index 2e929369848..a672723f33d --- a/docs/en/formats/values.md +++ b/docs/en/formats/values.md @@ -4,5 +4,5 @@ Prints every row in brackets. Rows are separated by commas. There is no comma af The minimum set of characters that you need to escape when passing data in Values ​​format: single quotes and backslashes. -This is the format that is used in `INSERT INTO t VALUES ...` but you can also use it for query result. +This is the format that is used in `INSERT INTO t VALUES ...`, but you can also use it for formatting query results. diff --git a/docs/en/formats/vertical.md b/docs/en/formats/vertical.md old mode 100644 new mode 100755 diff --git a/docs/en/formats/xml.md b/docs/en/formats/xml.md old mode 100644 new mode 100755 index f91adec9356..0da55875cc3 --- a/docs/en/formats/xml.md +++ b/docs/en/formats/xml.md @@ -35,7 +35,7 @@ XML format is suitable only for output, not for parsing. Example: 1549 - freeform photos + freeform photo 1480 diff --git a/docs/en/functions/arithmetic_functions.md b/docs/en/functions/arithmetic_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/array_functions.md b/docs/en/functions/array_functions.md old mode 100644 new mode 100755 index 493a465ac82..232f6a20427 --- a/docs/en/functions/array_functions.md +++ b/docs/en/functions/array_functions.md @@ -39,7 +39,7 @@ Accepts an empty array and returns a one-element array that is equal to the defa Returns an array of numbers from 0 to N-1. Just in case, an exception is thrown if arrays with a total length of more than 100,000,000 elements are created in a data block. -## array(x1, ...), operator \[x1, ...\] +## array(x1, ...), оператор \[x1, ...\] Creates an array from the function arguments. The arguments must be constants and have types that have the smallest common type. At least one argument must be passed, because otherwise it isn't clear which type of array to create. That is, you can't use this function to create an empty array (to do that, use the 'emptyArray\*' function described above). @@ -62,7 +62,6 @@ arrayConcat(arrays) ```sql SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res ``` - ``` ┌─res───────────┐ │ [1,2,3,4,5,6] │ @@ -203,7 +202,6 @@ arrayPopBack(array) ```sql SELECT arrayPopBack([1, 2, 3]) AS res ``` - ``` ┌─res───┐ │ [1,2] │ @@ -245,14 +243,13 @@ arrayPushBack(array, single_value) **Arguments** - `array` – Array. -- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../data_types/index.md#data_types)". +- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about ClickHouse data types, read the section "[Data types](../data_types/index.md#data_types)". **Example** ```sql SELECT arrayPushBack(['a'], 'b') AS res ``` - ``` ┌─res───────┐ │ ['a','b'] │ @@ -270,7 +267,7 @@ arrayPushFront(array, single_value) **Arguments** - `array` – Array. -- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about the types of data in ClickHouse, see "[Data types](../data_types/index.md#data_types)". +- `single_value` – A single value. Only numbers can be added to an array with numbers, and only strings can be added to an array of strings. When adding numbers, ClickHouse automatically sets the `single_value` type for the data type of the array. For more information about ClickHouse data types, read the section "[Data types](../data_types/index.md#data_types)". **Example** @@ -295,7 +292,7 @@ arraySlice(array, offset[, length]) **Arguments** - `array` – Array of data. -- `offset` – Offset from the edge of the array. A positive value indicates an offset on the left, and a negative value is an indent on the right. Numbering of the array items begins with 1. +- `offset` – Indent from the edge of the array. A positive value indicates an offset on the left, and a negative value is an indent on the right. Numbering of the array items begins with 1. - `length` - The length of the required slice. If you specify a negative value, the function returns an open slice `[offset, array_length - length)`. If you omit the value, the function returns the slice `[offset, the_end_of_array]`. **Example** @@ -303,7 +300,6 @@ arraySlice(array, offset[, length]) ```sql SELECT arraySlice([1, 2, 3, 4, 5], 2, 3) AS res ``` - ``` ┌─res─────┐ │ [2,3,4] │ diff --git a/docs/en/functions/array_join.md b/docs/en/functions/array_join.md old mode 100644 new mode 100755 index 6e18f8203c0..f94b2707f52 --- a/docs/en/functions/array_join.md +++ b/docs/en/functions/array_join.md @@ -28,4 +28,3 @@ SELECT arrayJoin([1, 2, 3] AS src) AS dst, 'Hello', src │ 3 │ Hello │ [1,2,3] │ └─────┴───────────┴─────────┘ ``` - diff --git a/docs/en/functions/bit_functions.md b/docs/en/functions/bit_functions.md old mode 100644 new mode 100755 index 523413f200a..c5a032aa5d6 --- a/docs/en/functions/bit_functions.md +++ b/docs/en/functions/bit_functions.md @@ -15,4 +15,3 @@ The result type is an integer with bits equal to the maximum bits of its argumen ## bitShiftLeft(a, b) ## bitShiftRight(a, b) - diff --git a/docs/en/functions/comparison_functions.md b/docs/en/functions/comparison_functions.md old mode 100644 new mode 100755 index e37642d42ed..9b95966ba84 --- a/docs/en/functions/comparison_functions.md +++ b/docs/en/functions/comparison_functions.md @@ -15,7 +15,7 @@ For example, you can't compare a date with a string. You have to use a function Strings are compared by bytes. A shorter string is smaller than all strings that start with it and that contain at least one more character. -Note: Up until version 1.1.54134, signed and unsigned numbers were compared the same way as in C++. In other words, you could get an incorrect result in cases like SELECT 9223372036854775807 > -1. This behavior changed in version 1.1.54134 and is now mathematically correct. +Note. Up until version 1.1.54134, signed and unsigned numbers were compared the same way as in C++. In other words, you could get an incorrect result in cases like SELECT 9223372036854775807 > -1. This behavior changed in version 1.1.54134 and is now mathematically correct. ## equals, a = b and a == b operator diff --git a/docs/en/functions/conditional_functions.md b/docs/en/functions/conditional_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/date_time_functions.md b/docs/en/functions/date_time_functions.md old mode 100644 new mode 100755 index da6ad9b4a7c..a7529e5f0e1 --- a/docs/en/functions/date_time_functions.md +++ b/docs/en/functions/date_time_functions.md @@ -79,6 +79,10 @@ Rounds down a date with time to the start of the minute. Rounds down a date with time to the start of the hour. +## toStartOfFifteenMinutes + +Rounds down the date with time to the start of the fifteen-minute interval. + Note: If you need to round a date with time to any other number of seconds, minutes, or hours, you can convert it into a number by using the toUInt32 function, then round the number using intDiv and multiplication, and convert it back using the toDateTime function. ## toStartOfHour diff --git a/docs/en/functions/encoding_functions.md b/docs/en/functions/encoding_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/ext_dict_functions.md b/docs/en/functions/ext_dict_functions.md old mode 100644 new mode 100755 index 949e805d9ab..002e2f55845 --- a/docs/en/functions/ext_dict_functions.md +++ b/docs/en/functions/ext_dict_functions.md @@ -18,18 +18,20 @@ For information on connecting and configuring external dictionaries, see "[Exter `dictGetT('dict_name', 'attr_name', id)` -- Get the value of the attr_name attribute from the dict_name dictionary using the 'id' key.`dict_name` and `attr_name` are constant strings.`id`must be UInt64. +- Get the value of the attr_name attribute from the dict_name dictionary using the 'id' key. +`dict_name` and `attr_name` are constant strings. +`id`must be UInt64. If there is no `id` key in the dictionary, it returns the default value specified in the dictionary description. ## dictGetTOrDefault `dictGetT('dict_name', 'attr_name', id, default)` -Similar to the functions dictGetT, but the default value is taken from the last argument of the function. +The same as the `dictGetT` functions, but the default value is taken from the function's last argument. ## dictIsIn -`dictIsIn ('dict_name', child_id, ancestor_id)` +`dictIsIn('dict_name', child_id, ancestor_id)` - For the 'dict_name' hierarchical dictionary, finds out whether the 'child_id' key is located inside 'ancestor_id' (or matches 'ancestor_id'). Returns UInt8. diff --git a/docs/en/functions/hash_functions.md b/docs/en/functions/hash_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/higher_order_functions.md b/docs/en/functions/higher_order_functions.md old mode 100644 new mode 100755 index ab9bdc50786..15cc40dead1 --- a/docs/en/functions/higher_order_functions.md +++ b/docs/en/functions/higher_order_functions.md @@ -73,7 +73,7 @@ Returns the index of the first element in the 'arr1' array for which 'func' retu ### arrayCumSum(\[func,\] arr1, ...) -Returns the cumulative sum of the array obtained from the original application of the 'func' function to each element in the 'arr' array. +Returns an array of partial sums of elements in the source array (a running sum). If the `func` function is specified, then the values of the array elements are converted by this function before summing. Example: @@ -86,4 +86,3 @@ SELECT arrayCumSum([1, 1, 1, 1]) AS res │ [1, 2, 3, 4] │ └──────────────┘ ``` - diff --git a/docs/en/functions/in_functions.md b/docs/en/functions/in_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/index.md b/docs/en/functions/index.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/ip_address_functions.md b/docs/en/functions/ip_address_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/json_functions.md b/docs/en/functions/json_functions.md old mode 100644 new mode 100755 index 1bf10e9cf0c..90a2ddc47dd --- a/docs/en/functions/json_functions.md +++ b/docs/en/functions/json_functions.md @@ -1,11 +1,11 @@ -# Functions for working with JSON. +# Functions for working with JSON In Yandex.Metrica, JSON is transmitted by users as session parameters. There are some special functions for working with this JSON. (Although in most of the cases, the JSONs are additionally pre-processed, and the resulting values are put in separate columns in their processed format.) All these functions are based on strong assumptions about what the JSON can be, but they try to do as little as possible to get the job done. The following assumptions are made: 1. The field name (function argument) must be a constant. -2. The field name is somehow canonically encoded in JSON. For example: `visitParamHas('{"abc":"def"}', 'abc') = 1`, but `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` +2. The field name is somehow canonically encoded in JSON. For example: `visitParamHas('{"abc":"def"}', 'abc') = 1`, но `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` 3. Fields are searched for on any nesting level, indiscriminately. If there are multiple matching fields, the first occurrence is used. 4. The JSON doesn't have space characters outside of string literals. @@ -47,7 +47,10 @@ Parses the string in double quotes. The value is unescaped. If unescaping failed Examples: ```text -visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0'visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺'visitParamExtractString('{"abc":"\\u263"}', 'abc') = ''visitParamExtractString('{"abc":"hello}', 'abc') = '' +visitParamExtractString('{"abc":"\\n\\u0000"}', 'abc') = '\n\0' +visitParamExtractString('{"abc":"\\u263a"}', 'abc') = '☺' +visitParamExtractString('{"abc":"\\u263"}', 'abc') = '' +visitParamExtractString('{"abc":"hello}', 'abc') = '' ``` There is currently no support for code points in the format `\uXXXX\uYYYY` that are not from the basic multilingual plane (they are converted to CESU-8 instead of UTF-8). diff --git a/docs/en/functions/logical_functions.md b/docs/en/functions/logical_functions.md old mode 100644 new mode 100755 index 4ef0fe5fd32..d396640a49d --- a/docs/en/functions/logical_functions.md +++ b/docs/en/functions/logical_functions.md @@ -11,4 +11,3 @@ Zero as an argument is considered "false," while any non-zero value is considere ## not, NOT operator ## xor - diff --git a/docs/en/functions/math_functions.md b/docs/en/functions/math_functions.md old mode 100644 new mode 100755 index d606c87a509..42e3f3e8018 --- a/docs/en/functions/math_functions.md +++ b/docs/en/functions/math_functions.md @@ -97,4 +97,3 @@ The arc tangent. ## pow(x, y) xy. - diff --git a/docs/en/functions/other_functions.md b/docs/en/functions/other_functions.md old mode 100644 new mode 100755 index befd94ecd4e..8a0063750fe --- a/docs/en/functions/other_functions.md +++ b/docs/en/functions/other_functions.md @@ -59,7 +59,8 @@ For elements in a nested data structure, the function checks for the existence o Allows building a unicode-art diagram. -`bar (x, min, max, width)` – Draws a band with a width proportional to (x - min) and equal to 'width' characters when x == max.`min, max` – Integer constants. The value must fit in Int64.`width` – Constant, positive number, may be a fraction. +`bar (x, min, max, width)` – Draws a band with a width proportional to (x - min) and equal to 'width' characters when x == max. +`min, max` – Integer constants. The value must fit in Int64.`width` – Constant, positive number, may be a fraction. The band is drawn with accuracy to one eighth of a symbol. @@ -137,7 +138,7 @@ Example: ```sql SELECT - transform(SearchEngineID, [2, 3], ['Yandex', 'Google'], 'Other' AS title, + transform(SearchEngineID, [2, 3], ['Yandex', 'Google'], 'Other') AS title, count() AS c FROM test.hits WHERE SearchEngineID != 0 diff --git a/docs/en/functions/random_functions.md b/docs/en/functions/random_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/rounding_functions.md b/docs/en/functions/rounding_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/splitting_merging_functions.md b/docs/en/functions/splitting_merging_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/string_functions.md b/docs/en/functions/string_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/string_replace_functions.md b/docs/en/functions/string_replace_functions.md old mode 100644 new mode 100755 index d3773504278..d70d8f404de --- a/docs/en/functions/string_replace_functions.md +++ b/docs/en/functions/string_replace_functions.md @@ -76,4 +76,3 @@ SELECT replaceRegexpAll('Hello, World!', '^', 'here: ') AS res │ here: Hello, World! │ └─────────────────────┘ ``` - diff --git a/docs/en/functions/string_search_functions.md b/docs/en/functions/string_search_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/type_conversion_functions.md b/docs/en/functions/type_conversion_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/url_functions.md b/docs/en/functions/url_functions.md old mode 100644 new mode 100755 diff --git a/docs/en/functions/ym_dict_functions.md b/docs/en/functions/ym_dict_functions.md old mode 100644 new mode 100755 index 540b5dd601a..7ba7e7012cf --- a/docs/en/functions/ym_dict_functions.md +++ b/docs/en/functions/ym_dict_functions.md @@ -21,7 +21,9 @@ All functions for working with regions have an optional argument at the end – Example: ```text -regionToCountry(RegionID) – Uses the default dictionary: /opt/geo/regions_hierarchy.txtregionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_hierarchy.txtregionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt +regionToCountry(RegionID) – Uses the default dictionary: /opt/geo/regions_hierarchy.txt +regionToCountry(RegionID, '') – Uses the default dictionary: /opt/geo/regions_hierarchy.txt +regionToCountry(RegionID, 'ua') – Uses the dictionary for the 'ua' key: /opt/geo/regions_hierarchy_ua.txt ``` ### regionToCity(id[, geobase]) @@ -33,7 +35,9 @@ Accepts a UInt32 number – the region ID from the Yandex geobase. If this regio Converts a region to an area (type 5 in the geobase). In every other way, this function is the same as 'regionToCity'. ```sql -SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua'))FROM system.numbersLIMIT 15 +SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua')) +FROM system.numbers +LIMIT 15 ``` ```text @@ -61,7 +65,9 @@ SELECT DISTINCT regionToName(regionToArea(toUInt32(number), 'ua'))FROM system.nu Converts a region to a federal district (type 4 in the geobase). In every other way, this function is the same as 'regionToCity'. ```sql -SELECT DISTINCT regionToName(regionToDistrict(toUInt32(number), 'ua'))FROM system.numbersLIMIT 15 +SELECT DISTINCT regionToName(regionToDistrict(toUInt32(number), 'ua')) +FROM system.numbers +LIMIT 15 ``` ```text diff --git a/docs/en/getting_started/example_datasets/amplab_benchmark.md b/docs/en/getting_started/example_datasets/amplab_benchmark.md old mode 100644 new mode 100755 diff --git a/docs/en/getting_started/example_datasets/criteo.md b/docs/en/getting_started/example_datasets/criteo.md old mode 100644 new mode 100755 index 9b59d6e5f3d..3c60a68f430 --- a/docs/en/getting_started/example_datasets/criteo.md +++ b/docs/en/getting_started/example_datasets/criteo.md @@ -66,6 +66,8 @@ CREATE TABLE criteo Transform data from the raw log and put it in the second table: ```sql -INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log;DROP TABLE criteo_log; +INSERT INTO criteo SELECT date, clicked, int1, int2, int3, int4, int5, int6, int7, int8, int9, int10, int11, int12, int13, reinterpretAsUInt32(unhex(cat1)) AS icat1, reinterpretAsUInt32(unhex(cat2)) AS icat2, reinterpretAsUInt32(unhex(cat3)) AS icat3, reinterpretAsUInt32(unhex(cat4)) AS icat4, reinterpretAsUInt32(unhex(cat5)) AS icat5, reinterpretAsUInt32(unhex(cat6)) AS icat6, reinterpretAsUInt32(unhex(cat7)) AS icat7, reinterpretAsUInt32(unhex(cat8)) AS icat8, reinterpretAsUInt32(unhex(cat9)) AS icat9, reinterpretAsUInt32(unhex(cat10)) AS icat10, reinterpretAsUInt32(unhex(cat11)) AS icat11, reinterpretAsUInt32(unhex(cat12)) AS icat12, reinterpretAsUInt32(unhex(cat13)) AS icat13, reinterpretAsUInt32(unhex(cat14)) AS icat14, reinterpretAsUInt32(unhex(cat15)) AS icat15, reinterpretAsUInt32(unhex(cat16)) AS icat16, reinterpretAsUInt32(unhex(cat17)) AS icat17, reinterpretAsUInt32(unhex(cat18)) AS icat18, reinterpretAsUInt32(unhex(cat19)) AS icat19, reinterpretAsUInt32(unhex(cat20)) AS icat20, reinterpretAsUInt32(unhex(cat21)) AS icat21, reinterpretAsUInt32(unhex(cat22)) AS icat22, reinterpretAsUInt32(unhex(cat23)) AS icat23, reinterpretAsUInt32(unhex(cat24)) AS icat24, reinterpretAsUInt32(unhex(cat25)) AS icat25, reinterpretAsUInt32(unhex(cat26)) AS icat26 FROM criteo_log; + +DROP TABLE criteo_log; ``` diff --git a/docs/en/getting_started/example_datasets/nyc_taxi.md b/docs/en/getting_started/example_datasets/nyc_taxi.md old mode 100644 new mode 100755 index 11ed81d1a43..a9f04f595d1 --- a/docs/en/getting_started/example_datasets/nyc_taxi.md +++ b/docs/en/getting_started/example_datasets/nyc_taxi.md @@ -1,8 +1,8 @@ -# Data about New York taxis +# New York Taxi data -## How to import raw data +## How to import the raw data -See and for description of the dataset and loading instructions. +See and for the description of the dataset and instructions for downloading. Downloading will result in about 227 GB of uncompressed data in CSV files. The download takes about an hour over a 1 Gbit connection (parallel downloading from s3.amazonaws.com recovers at least half of a 1 Gbit channel). Some of the files might not download fully. Check the file sizes and re-download any that seem doubtful. @@ -301,19 +301,14 @@ SELECT passenger_count, toYear(pickup_date) AS year, count(*) FROM trips_mergetr Q4: ```sql -SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*) -FROM trips_mergetree -GROUP BY passenger_count, year, distance -ORDER BY year, count(*) DESC +SELECT passenger_count, toYear(pickup_date) AS year, round(trip_distance) AS distance, count(*)FROM trips_mergetreeGROUP BY passenger_count, year, distanceORDER BY year, count(*) DESC ``` 3.593 seconds. The following server was used: -Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total, -128 GiB RAM, -8x6 TB HD on hardware RAID-5 +Two Intel(R) Xeon(R) CPU E5-2650 v2 @ 2.60GHz, 16 physical kernels total,128 GiB RAM,8x6 TB HD on hardware RAID-5 Execution time is the best of three runsBut starting from the second run, queries read data from the file system cache. No further caching occurs: the data is read out and processed in each run. diff --git a/docs/en/getting_started/example_datasets/ontime.md b/docs/en/getting_started/example_datasets/ontime.md old mode 100644 new mode 100755 diff --git a/docs/en/getting_started/example_datasets/star_schema.md b/docs/en/getting_started/example_datasets/star_schema.md old mode 100644 new mode 100755 index 8807de3e670..664ba59f48c --- a/docs/en/getting_started/example_datasets/star_schema.md +++ b/docs/en/getting_started/example_datasets/star_schema.md @@ -1,4 +1,4 @@ -# Star scheme +# Star Schema Benchmark Compiling dbgen: @@ -82,3 +82,4 @@ Downloading data (change 'customer' to 'customerd' in the distributed version): cat customer.tbl | sed 's/$/2000-01-01/' | clickhouse-client --query "INSERT INTO customer FORMAT CSV" cat lineorder.tbl | clickhouse-client --query "INSERT INTO lineorder FORMAT CSV" ``` + diff --git a/docs/en/getting_started/example_datasets/wikistat.md b/docs/en/getting_started/example_datasets/wikistat.md old mode 100644 new mode 100755 index 6cbc3b15561..fee0a56b52c --- a/docs/en/getting_started/example_datasets/wikistat.md +++ b/docs/en/getting_started/example_datasets/wikistat.md @@ -20,7 +20,7 @@ CREATE TABLE wikistat Loading data: ```bash -for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sS "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt +for i in {2007..2016}; do for j in {01..12}; do echo $i-$j >&2; curl -sSL "http://dumps.wikimedia.org/other/pagecounts-raw/$i/$i-$j/" | grep -oE 'pagecounts-[0-9]+-[0-9]+\.gz'; done; done | sort | uniq | tee links.txt cat links.txt | while read link; do wget http://dumps.wikimedia.org/other/pagecounts-raw/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1/')/$(echo $link | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})[0-9]{2}-[0-9]+\.gz/\1-\2/')/$link; done ls -1 /opt/wikistat/ | grep gz | while read i; do echo $i; gzip -cd /opt/wikistat/$i | ./wikistat-loader --time="$(echo -n $i | sed -r 's/pagecounts-([0-9]{4})([0-9]{2})([0-9]{2})-([0-9]{2})([0-9]{2})([0-9]{2})\.gz/\1-\2-\3 \4-00-00/')" | clickhouse-client --query="INSERT INTO wikistat FORMAT TabSeparated"; done ``` diff --git a/docs/en/getting_started/index.md b/docs/en/getting_started/index.md old mode 100644 new mode 100755 index 42fc1c75551..07d0d91a224 --- a/docs/en/getting_started/index.md +++ b/docs/en/getting_started/index.md @@ -16,14 +16,15 @@ The terminal must use UTF-8 encoding (the default in Ubuntu). For testing and development, the system can be installed on a single server or on a desktop computer. -### Installing from packages Debian/Ubuntu +### Installing from packages In `/etc/apt/sources.list` (or in a separate `/etc/apt/sources.list.d/clickhouse.list` file), add the repository: ```text -deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ +deb http://repo.yandex.ru/clickhouse/trusty stable main ``` +On other versions of Ubuntu, replace `trusty` with `xenial` or `precise`. If you want to use the most recent test version, replace 'stable' with 'testing'. Then run: @@ -34,7 +35,10 @@ sudo apt-get update sudo apt-get install clickhouse-client clickhouse-server-common ``` -You can also download and install packages manually from here: +You can also download and install packages manually from here: + + + ClickHouse contains access restriction settings. They are located in the 'users.xml' file (next to 'config.xml'). By default, access is allowed from anywhere for the 'default' user, without a password. See 'user/default/networks'. @@ -100,7 +104,8 @@ clickhouse-client ``` The default parameters indicate connecting with localhost:9000 on behalf of the user 'default' without a password. -The client can be used for connecting to a remote server. Example: +The client can be used for connecting to a remote server. +Example: ```bash clickhouse-client --host=example.com @@ -132,3 +137,4 @@ SELECT 1 **Congratulations, the system works!** To continue experimenting, you can try to download from the test data sets. + diff --git a/docs/en/index.md b/docs/en/index.md old mode 100644 new mode 100755 index 586c18297a8..72efa70802b --- a/docs/en/index.md +++ b/docs/en/index.md @@ -39,7 +39,7 @@ We'll say that the following is true for the OLAP (online analytical processing) - Data is updated in fairly large batches (> 1000 rows), not by single rows; or it is not updated at all. - Data is added to the DB but is not modified. - For reads, quite a large number of rows are extracted from the DB, but only a small subset of columns. -- Tables are "wide", meaning they contain a large number of columns. +- Tables are "wide," meaning they contain a large number of columns. - Queries are relatively rare (usually hundreds of queries per server or less per second). - For simple queries, latencies around 50 ms are allowed. - Column values are fairly small: numbers and short strings (for example, 60 bytes per URL). diff --git a/docs/en/interfaces/cli.md b/docs/en/interfaces/cli.md old mode 100644 new mode 100755 index 4fd998fed66..76549b46b36 --- a/docs/en/interfaces/cli.md +++ b/docs/en/interfaces/cli.md @@ -6,7 +6,9 @@ To work from the command line, you can use ` clickhouse-client`: $ clickhouse-client ClickHouse client version 0.0.26176. Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.26176.:) +Connected to ClickHouse server version 0.0.26176. + +:) ``` The client supports command-line options and configuration files. For more information, see "[Configuring](#interfaces_cli_configuration)". @@ -29,7 +31,6 @@ _EOF cat file.csv | clickhouse-client --database=test --query="INSERT INTO test FORMAT CSV"; ``` -In batch mode, the default data format is TabSeparated. You can set the format in the FORMAT clause of the query. By default, you can only process a single query in batch mode. To make multiple queries from a "script," use the --multiquery parameter. This works for all queries except INSERT. Query results are output consecutively without additional separators. Similarly, to process a large number of queries, you can run 'clickhouse-client' for each query. Note that it may take tens of milliseconds to launch the 'clickhouse-client' program. @@ -64,7 +65,7 @@ The command-line client allows passing external data (external temporary tables) -## Configure +## Configuring You can pass parameters to `clickhouse-client` (all parameters have a default value) using: diff --git a/docs/en/interfaces/http_interface.md b/docs/en/interfaces/http_interface.md old mode 100644 new mode 100755 index 5c989a59d65..38a70feef46 --- a/docs/en/interfaces/http_interface.md +++ b/docs/en/interfaces/http_interface.md @@ -37,7 +37,8 @@ Date: Fri, 16 Nov 2012 19:21:50 GMT 1 ``` -As you can see, curl is somewhat inconvenient in that spaces must be URL escaped.Although wget escapes everything itself, we don't recommend using it because it doesn't work well over HTTP 1.1 when using keep-alive and Transfer-Encoding: chunked. +As you can see, curl is somewhat inconvenient in that spaces must be URL escaped. +Although wget escapes everything itself, we don't recommend using it because it doesn't work well over HTTP 1.1 when using keep-alive and Transfer-Encoding: chunked. ```bash $ echo 'SELECT 1' | curl 'http://localhost:8123/' --data-binary @- @@ -130,15 +131,11 @@ POST 'http://localhost:8123/?query=DROP TABLE t' For successful requests that don't return a data table, an empty response body is returned. -You can use compression when transmitting data. +You can use compression when transmitting data. The compressed data has a non-standard format, and you will need to use the special compressor program to work with it (sudo apt-get install compressor-metrika-yandex). -For using ClickHouse internal compression format, and you will need to use the special clickhouse-compressor program to work with it (installed as a part of clickhouse-client package). If you specified 'compress=1' in the URL, the server will compress the data it sends you. If you specified 'decompress=1' in the URL, the server will decompress the same data that you pass in the POST method. -Also standard gzip-based HTTP compression can be used. To send gzip compressed POST data just add `Content-Encoding: gzip` to request headers, and gzip POST body. -To get response compressed, you need to add `Accept-Encoding: gzip` to request headers, and turn on ClickHouse setting called `enable_http_compression`. - You can use this to reduce network traffic when transmitting a large amount of data, or for creating dumps that are immediately compressed. You can use the 'database' URL parameter to specify the default database. @@ -194,11 +191,7 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 For information about other parameters, see the section "SET". -You can use ClickHouse sessions in the HTTP protocol. To do this, you need to specify the `session_id` GET parameter in HTTP request. You can use any alphanumeric string as a session_id. By default session will be timed out after 60 seconds of inactivity. You can change that by setting `default_session_timeout` in server config file, or by adding GET parameter `session_timeout`. You can also check the status of the session by using GET parameter `session_check=1`. When using sessions you can't run 2 queries with the same session_id simultaneously. - -You can get the progress of query execution in X-ClickHouse-Progress headers, by enabling setting send_progress_in_http_headers. - -Running query are not aborted automatically after closing HTTP connection. Parsing and data formatting are performed on the server side, and using the network might be ineffective. +In contrast to the native interface, the HTTP interface does not support the concept of sessions or session settings, does not allow aborting a query (to be exact, it allows this in only a few cases), and does not show the progress of query processing. Parsing and data formatting are performed on the server side, and using the network might be ineffective. The optional 'query_id' parameter can be passed as the query ID (any string). For more information, see the section "Settings, replace_running_query". The optional 'quota_key' parameter can be passed as the quota key (any string). For more information, see the section "Quotas". @@ -220,3 +213,4 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa ``` Use buffering to avoid situations where a query processing error occurred after the response code and HTTP headers were sent to the client. In this situation, an error message is written at the end of the response body, and on the client side, the error can only be detected at the parsing stage. + diff --git a/docs/en/interfaces/index.md b/docs/en/interfaces/index.md old mode 100644 new mode 100755 index 3e3e3df4853..e43f4474271 --- a/docs/en/interfaces/index.md +++ b/docs/en/interfaces/index.md @@ -2,4 +2,5 @@ # Interfaces -To explore the system's capabilities, download data to tables, or make manual queries, use the clickhouse-client program. \ No newline at end of file +To explore the system's capabilities, download data to tables, or make manual queries, use the clickhouse-client program. + diff --git a/docs/en/interfaces/jdbc.md b/docs/en/interfaces/jdbc.md old mode 100644 new mode 100755 diff --git a/docs/en/interfaces/tcp.md b/docs/en/interfaces/tcp.md old mode 100644 new mode 100755 diff --git a/docs/en/interfaces/third-party_client_libraries.md b/docs/en/interfaces/third-party_client_libraries.md old mode 100644 new mode 100755 index cc8ff1f4307..8437be23b99 --- a/docs/en/interfaces/third-party_client_libraries.md +++ b/docs/en/interfaces/third-party_client_libraries.md @@ -2,7 +2,7 @@ There are libraries for working with ClickHouse for: -- Python: +- Python - [infi.clickhouse_orm](https://github.com/Infinidat/infi.clickhouse_orm) - [sqlalchemy-clickhouse](https://github.com/cloudflare/sqlalchemy-clickhouse) - [clickhouse-driver](https://github.com/mymarilyn/clickhouse-driver) diff --git a/docs/en/interfaces/third-party_gui.md b/docs/en/interfaces/third-party_gui.md old mode 100644 new mode 100755 diff --git a/docs/en/introduction/distinctive_features.md b/docs/en/introduction/distinctive_features.md old mode 100644 new mode 100755 index 3927405579f..59853b8e202 --- a/docs/en/introduction/distinctive_features.md +++ b/docs/en/introduction/distinctive_features.md @@ -1,10 +1,10 @@ # Distinctive features of ClickHouse -## True column-oriented DBMS. +## True column-oriented DBMS -In a true column-oriented DBMS, there isn't any "garbage" stored with the values. For example, constant-length values must be supported, to avoid storing their length "number" next to the values. As an example, a billion UInt8-type values should actually consume around 1 GB uncompressed, or this will strongly affect the CPU use. It is very important to store data compactly (without any "garbage") even when uncompressed, since the speed of decompression (CPU usage) depends mainly on the volume of uncompressed data. +In a true column-oriented DBMS, there isn't any "garbage" stored with the values. Among other things, this means that constant-length values must be supported, to avoid storing their length "number" next to the values. As an example, a billion UInt8-type values should actually consume around 1 GB uncompressed, or this will strongly affect the CPU use. It is very important to store data compactly (without any "garbage") even when uncompressed, since the speed of decompression (CPU usage) depends mainly on the volume of uncompressed data. -This is worth noting because there are systems that can store values of separate columns separately, but that can't effectively process analytical queries due to their optimization for other scenarios. Example are HBase, BigTable, Cassandra, and HyperTable. In these systems, you will get throughput around a hundred thousand rows per second, but not hundreds of millions of rows per second. +This is worth noting because there are systems that can store values of separate columns separately, but that can't effectively process analytical queries due to their optimization for other scenarios. Examples are HBase, BigTable, Cassandra, and HyperTable. In these systems, you will get throughput around a hundred thousand rows per second, but not hundreds of millions of rows per second. Also note that ClickHouse is a DBMS, not a single database. ClickHouse allows creating tables and databases in runtime, loading data, and running queries without reconfiguring and restarting the server. @@ -12,15 +12,15 @@ Also note that ClickHouse is a DBMS, not a single database. ClickHouse allows cr Some column-oriented DBMSs (InfiniDB CE and MonetDB) do not use data compression. However, data compression really improves performance. -## Disk storage of data. +## Disk storage of data Many column-oriented DBMSs (such as SAP HANA and Google PowerDrill) can only work in RAM. But even on thousands of servers, the RAM is too small for storing all the pageviews and sessions in Yandex.Metrica. -## Parallel processing on multiple cores. +## Parallel processing on multiple cores Large queries are parallelized in a natural way. -## Distributed processing on multiple servers. +## Distributed processing on multiple servers Almost none of the columnar DBMSs listed above have support for distributed processing. In ClickHouse, data can reside on different shards. Each shard can be a group of replicas that are used for fault tolerance. The query is processed on all the shards in parallel. This is transparent for the user. @@ -30,12 +30,12 @@ In ClickHouse, data can reside on different shards. Each shard can be a group of If you are familiar with standard SQL, we can't really talk about SQL support. All the functions have different names. However, this is a declarative query language based on SQL that can't be differentiated from SQL in many instances. -Support for JOINs. Subqueries are supported in FROM, IN, and JOIN clauses, as well as scalar subqueries. +JOINs are supported. Subqueries are supported in FROM, IN, and JOIN clauses, as well as scalar subqueries. Dependent subqueries are not supported. ## Vector engine -Data is not only stored by columns, but is processed by vectors – parts of columns. This allows us to achieve high CPU performance. +Data is not only stored by columns, but is processed by vectors (parts of columns). This allows us to achieve high CPU performance. ## Real-time data updates @@ -43,13 +43,13 @@ ClickHouse supports primary key tables. In order to quickly perform queries on t ## Indexes -Having a primary key allows, for example, extracting data for specific clients (Metrica counters) for a specific time range, with low latency less than several dozen milliseconds. +Having a primary key makes it possible to extract data for specific clients (for instance, Yandex.Metrica tracking tags) for a specific time range, with low latency less than several dozen milliseconds. ## Suitable for online queries This lets us use the system as the back-end for a web interface. Low latency means queries can be processed without delay, while the Yandex.Metrica interface page is loading. In other words, in online mode. -## Support for approximated calculations. +## Support for approximated calculations 1. The system contains aggregate functions for approximated calculation of the number of various values, medians, and quantiles. 2. Supports running a query based on a part (sample) of data and getting an approximated result. In this case, proportionally less data is retrieved from the disk. diff --git a/docs/en/introduction/features_considered_disadvantages.md b/docs/en/introduction/features_considered_disadvantages.md old mode 100644 new mode 100755 diff --git a/docs/en/introduction/index.md b/docs/en/introduction/index.md old mode 100644 new mode 100755 index e10b99d0138..3d07efe555d --- a/docs/en/introduction/index.md +++ b/docs/en/introduction/index.md @@ -1 +1,2 @@ # Introduction + diff --git a/docs/en/introduction/performance.md b/docs/en/introduction/performance.md old mode 100644 new mode 100755 diff --git a/docs/en/introduction/possible_silly_questions.md b/docs/en/introduction/possible_silly_questions.md old mode 100644 new mode 100755 index 36363ebe247..cf7b2c48032 --- a/docs/en/introduction/possible_silly_questions.md +++ b/docs/en/introduction/possible_silly_questions.md @@ -1,8 +1,8 @@ -# Everything you were afraid to ask +# Questions you were afraid to ask ## Why not use something like MapReduce? -We can refer to systems like map-reduce as distributed computing systems in which the reduce operation is based on distributed sorting. In this sense, they include Hadoop and YT (Yandex proprietary technology). +We can refer to systems like map-reduce as distributed computing systems in which the reduce operation is based on distributed sorting. In this sense, they include Hadoop, and YT (YT is developed at Yandex for internal use). These systems aren't appropriate for online queries due to their high latency. In other words, they can't be used as the back-end for a web interface. These types of systems aren't useful for real-time data updates. diff --git a/docs/en/introduction/ya_metrika_task.md b/docs/en/introduction/ya_metrika_task.md old mode 100644 new mode 100755 index 6a488be9b5f..10f45f061d6 --- a/docs/en/introduction/ya_metrika_task.md +++ b/docs/en/introduction/ya_metrika_task.md @@ -1,6 +1,6 @@ -# The Yandex.Metrica task +# Yandex.Metrica use case -ClickHouse currently powers [ Yandex.Metrica](https://metrika.yandex.ru/), [ the second largest platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all) for Web Analytics. With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows you generating custom reports on the fly directly from non-aggregated data. +ClickHouse currently powers [Yandex.Metrica](https://metrika.yandex.ru/), [the second largest web analytics platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all). With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows you generating custom reports on the fly directly from non-aggregated data. We need to get custom reports based on hits and sessions, with custom segments set by the user. Data for the reports is updated in real-time. Queries must be run immediately (in online mode). We must be able to build reports for any time period. Complex aggregates must be calculated, such as the number of unique visitors. At this time (April 2014), Yandex.Metrica receives approximately 12 billion events (pageviews and mouse clicks) daily. All these events must be stored in order to build custom reports. A single query may require scanning hundreds of millions of rows over a few seconds, or millions of rows in no more than a few hundred milliseconds. diff --git a/docs/en/operations/access_rights.md b/docs/en/operations/access_rights.md old mode 100644 new mode 100755 index 9879dab9a99..1c72bf13b3e --- a/docs/en/operations/access_rights.md +++ b/docs/en/operations/access_rights.md @@ -2,14 +2,14 @@ Users and access rights are set up in the user config. This is usually `users.xml`. -Users are recorded in the 'users' section. We'll look at a fragment of the `users.xml` file: +Users are recorded in the 'users' section. Here is a fragment of the `users.xml` file: ```xml - - - + How to generate a decent password: Execute: PASSWORD=$(base64 < /dev/urandom | head -c8); echo "$PASSWORD"; echo -n "$PASSWORD" | sha256sum | tr -d '-' + The first line has the password and the second line has the corresponding SHA256. + --> + + - - - - default - + + ::/0 + + --> + + + + default + default - - - - - - - web - default - - test + + + + + + + web + default + + test ``` You can see a declaration from two users: `default`and`web`. We added the `web` user separately. -The `default` user is chosen in cases when the username is not passed. The `default` user is also used for distributed query processing, if the configuration of the server or cluster doesn't specify `user` and `password` (see the section on the [Distributed](../table_engines/distributed.md#distributed_distributed) engine). +The `default` user is chosen in cases when the username is not passed. The `default` user is also used for distributed query processing, if the configuration of the server or cluster doesn't specify the `user` and `password` (see the section on the [Distributed](../table_engines/distributed.md#table_engines-distributed) engine). The user that is used for exchanging information between servers combined in a cluster must not have substantial restrictions or quotas – otherwise, distributed queries will fail. diff --git a/docs/en/operations/configuration_files.md b/docs/en/operations/configuration_files.md old mode 100644 new mode 100755 index c3122617bf1..52e9e10ffea --- a/docs/en/operations/configuration_files.md +++ b/docs/en/operations/configuration_files.md @@ -14,7 +14,7 @@ If `replace` is specified, it replaces the entire element with the specified one If ` remove` is specified, it deletes the element. -The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the config in the [include_from](server_settings/settings.md#server_settings-include_from) element. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in ` incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional = "true"` attribute (for example, settings for [ macros](server_settings/settings.md#server_settings-macros)). +The config can also define "substitutions". If an element has the `incl` attribute, the corresponding substitution from the file will be used as the value. By default, the path to the file with substitutions is `/etc/metrika.xml`. This can be changed in the [include_from](server_settings/settings.md#server_settings-include_from) element in the server config. The substitution values are specified in `/yandex/substitution_name` elements in this file. If a substitution specified in ` incl` does not exist, it is recorded in the log. To prevent ClickHouse from logging missing substitutions, specify the `optional="true"` attribute (for example, settings for [macros]()server_settings/settings.md#server_settings-macros)). Substitutions can also be performed from ZooKeeper. To do this, specify the attribute `from_zk = "/path/to/node"`. The element value is replaced with the contents of the node at ` /path/to/node` in ZooKeeper. You can also put an entire XML subtree on the ZooKeeper node and it will be fully inserted into the source element. diff --git a/docs/en/operations/index.md b/docs/en/operations/index.md old mode 100644 new mode 100755 index eb90f937cff..0ff38af8086 --- a/docs/en/operations/index.md +++ b/docs/en/operations/index.md @@ -1 +1,2 @@ -# Operation +# Usage + diff --git a/docs/en/operations/quotas.md b/docs/en/operations/quotas.md old mode 100644 new mode 100755 index d7b1a61ce7f..fb1238b257d --- a/docs/en/operations/quotas.md +++ b/docs/en/operations/quotas.md @@ -18,10 +18,10 @@ Let's look at the section of the 'users.xml' file that defines quotas. - + - 3600 - + 3600 + 0 0 @@ -39,21 +39,19 @@ The resource consumption calculated for each interval is output to the server lo - - 3600 + + 3600 + 1000 + 100 + 1000000000 + 100000000000 + 900 + - 1000 - 100 - 1000000000 - 100000000000 - 900 - - - - 86400 - - 10000 - 1000 + + 86400 + 10000 + 1000 5000000000 500000000000 7200 @@ -89,7 +87,7 @@ Quotas can use the "quota key" feature in order to report on resources for multi Using keys makes sense only if quota_key is transmitted by the program, not by a user. You can also write so the IP address is used as the quota key.(But keep in mind that users can change the IPv6 address fairly easily.) - --> + --> ``` @@ -98,4 +96,3 @@ The quota is assigned to users in the 'users' section of the config. See the sec For distributed query processing, the accumulated amounts are stored on the requestor server. So if the user goes to another server, the quota there will "start over". When the server is restarted, quotas are reset. - diff --git a/docs/en/operations/server_settings/index.md b/docs/en/operations/server_settings/index.md old mode 100644 new mode 100755 index 2293e86f5c7..208deec710c --- a/docs/en/operations/server_settings/index.md +++ b/docs/en/operations/server_settings/index.md @@ -9,3 +9,4 @@ These settings are stored in the ` config.xml` file on the ClickHouse server. Other settings are described in the "[Settings](../settings/index.md#settings)" section. Before studying the settings, read the [Configuration files](../configuration_files.md#configuration_files) section and note the use of substitutions (the `incl` and `optional` attributes). + diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md old mode 100644 new mode 100755 index 8818f8ec932..e1575df2f88 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -67,7 +67,7 @@ ClickHouse checks ` min_part_size` and ` min_part_size_ratio` and processes th The default database. -Use a [ SHOW DATABASES](../../query_language/queries.md#query_language_queries_show_databases) query to get a list of databases. +To get a list of databases, use the [ SHOW DATABASES]( query./../query_language/queries.md#query_language_queries_show_databases). **Example** @@ -81,7 +81,7 @@ Use a [ SHOW DATABASES](../../query_language/queries.md#query_language_queries_s Default settings profile. -Settings profiles are located in the file specified in the [user_config](#server_settings-users_config) parameter. +Settings profiles are located in the file specified in the parameter [user_config](#server_settings-users_config). **Example** @@ -100,7 +100,7 @@ Path: - Specify the absolute path or the path relative to the server config file. - The path can contain wildcards \* and ?. -See also "[External dictionaries](../../dicts/external_dicts.md#dicts-external_dicts)". +See also "[External dictionaries]("./../dicts/external_dicts.md#dicts-external_dicts)". **Example** @@ -130,12 +130,12 @@ The default is ` true`. ## format_schema_path -The path to the directory with the schemas for the input data, such as schemas for the [ CapnProto](../../formats/capnproto.md#format_capnproto) format. +The path to the directory with the schemes for the input data, such as schemas for the [CapnProto](../../formats/capnproto.md#format_capnproto) format. **Example** ```xml - + format_schemas/ ``` @@ -179,7 +179,7 @@ You can configure multiple `` clauses. For instance, you can use this Settings for thinning data for Graphite. -For more details, see [ GraphiteMergeTree](../../table_engines/graphitemergetree.md#table_engines-graphitemergetree). +For more information, see [GraphiteMergeTree](../../table_engines/graphitemergetree.md#table_engines-graphitemergetree). **Example** @@ -241,7 +241,7 @@ Opens `https://tabix.io/` when accessing ` http://localhost: http_port`. The path to the file with substitutions. -For details, see the section "[Configuration files](../configuration_files.md#configuration_files)". +For more information, see the section "[Configuration files](../configuration_files.md#configuration_files)". **Example** @@ -298,7 +298,8 @@ Restriction on hosts that requests can come from. If you want the server to answ Examples: ```xml -::1127.0.0.1 +::1 +127.0.0.1 ``` @@ -347,7 +348,7 @@ For more information, see the section "[Creating replicated tables](../../table_ ## mark_cache_size -Approximate size (in bytes) of the cache of "marks" used by [ MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) engines. +Approximate size (in bytes) of the cache of "marks" used by [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) family. The cache is shared for the server and memory is allocated as needed. The cache size must be at least 5368709120. @@ -403,7 +404,7 @@ We recommend using this option in Mac OS X, since the ` getrlimit()` function re Restriction on deleting tables. -If the size of a [ MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) type table exceeds `max_table_size_to_drop` (in bytes), you can't delete it using a DROP query. +If the size of a [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) type table exceeds `max_table_size_to_drop` (in bytes), you can't delete it using a DROP query. If you still need to delete the table without restarting the ClickHouse server, create the ` /flags/force_drop_table` file and run the DROP query. @@ -439,17 +440,17 @@ For more information, see the MergeTreeSettings.h header file. SSL client/server configuration. -Support for SSL is provided by the `` libpoco`` library. The description of the interface is in the [ SSLManager.h file.](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h) +Support for SSL is provided by the `` libpoco`` library. The interface is described in the file [SSLManager.h](https://github.com/yandex/ClickHouse/blob/master/contrib/libpoco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h) Keys for server/client settings: - privateKeyFile – The path to the file with the secret key of the PEM certificate. The file may contain a key and certificate at the same time. - certificateFile – The path to the client/server certificate file in PEM format. You can omit it if `` privateKeyFile`` contains the certificate. - caConfig – The path to the file or directory that contains trusted root certificates. -- verificationMode – The method for checking the node's certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Acceptable values: ``none``, ``relaxed``, ``strict``, ``once``. +- verificationMode – The method for checking the node's certificates. Details are in the description of the [Context](https://github.com/yandex/ClickHouse/blob/master/contrib/libpoco/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: ``none``, ``relaxed``, ``strict``, ``once``. - verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `` true``, `` false``. | -- cipherList - Supported OpenSSL-ciphers. For example: `` ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH``. +- cipherList - Поддерживаемые OpenSSL-шифры. For example: `` ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH``. - cacheSessions – Enables or disables caching sessions. Must be used in combination with ``sessionIdContext``. Acceptable values: `` true``, `` false``. - sessionIdContext – A unique set of random characters that the server appends to each generated identifier. The length of the string must not exceed ``SSL_MAX_SSL_SESSION_ID_LENGTH``. This parameter is always recommended, since it helps avoid problems both if the server caches the session and if the client requested caching. Default value: ``${application.name}``. - sessionCacheSize – The maximum number of sessions that the server caches. Default value: 1024\*20. 0 – Unlimited sessions. @@ -498,7 +499,7 @@ Keys for server/client settings: ## part_log -Logging events that are associated with the [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) data type. For instance, adding or merging data. You can use the log to simulate merge algorithms and compare their characteristics. You can visualize the merge process. +Logging events that are associated with [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) data. For instance, adding or merging data. You can use the log to simulate merge algorithms and compare their characteristics. You can visualize the merge process. Queries are logged in the ClickHouse table, not in a separate file. @@ -518,7 +519,7 @@ Use the following parameters to configure logging: - database – Name of the database. - table – Name of the table. -- partition_by - Sets the [custom partition key](../../table_engines/custom_partitioning_key.md#custom-partitioning-key). +- partition_by – Sets a [custom partitioning key](../../table_engines/custom_partitioning_key.md#custom-partitioning-key). - flush_interval_milliseconds – Interval for flushing data from memory to the disk. **Example** @@ -562,7 +563,7 @@ Use the following parameters to configure logging: - database – Name of the database. - table – Name of the table. -- partition_by - Sets the [custom partition key](../../table_engines/custom_partitioning_key.md#custom-partitioning-key). +- partition_by – Sets a [custom partitioning key](../../table_engines/custom_partitioning_key.md#custom-partitioning-key). - flush_interval_milliseconds – Interval for flushing data from memory to the disk. If the table doesn't exist, ClickHouse will create it. If the structure of the query log changed when the ClickHouse server was updated, the table with the old structure is renamed, and a new table is created automatically. @@ -584,7 +585,7 @@ If the table doesn't exist, ClickHouse will create it. If the structure of the q Configuration of clusters used by the Distributed table engine. -For more information, see the section "[Duplicated table engine](../../table_engines/distributed.md#table_engines-distributed)". +For more information, see the section "[Table engines/Distributed](../../table_engines/distributed.md#table_engines-distributed)". **Example** @@ -644,7 +645,7 @@ The end slash is mandatory. ## uncompressed_cache_size -Cache size (in bytes) for uncompressed data used by table engines from the [ MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) family. +Cache size (in bytes) for uncompressed data used by table engines from the [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) family. There is one shared cache for the server. Memory is allocated on demand. The cache is used if the option [use_uncompressed_cache](../settings/settings.md#settings-use_uncompressed_cache) is enabled. diff --git a/docs/en/operations/settings/index.md b/docs/en/operations/settings/index.md old mode 100644 new mode 100755 index 0e967a4c081..0c5ca5d5171 --- a/docs/en/operations/settings/index.md +++ b/docs/en/operations/settings/index.md @@ -9,9 +9,9 @@ Ways to configure settings, in order of priority: - Settings in the server config file. - Set via user profiles. + Settings from user profiles. -- For the session. +- Session settings. Send ` SET setting=value` from the ClickHouse console client in interactive mode. Similarly, you can use ClickHouse sessions in the HTTP protocol. To do this, you need to specify the `session_id` HTTP parameter. diff --git a/docs/en/operations/settings/query_complexity.md b/docs/en/operations/settings/query_complexity.md old mode 100644 new mode 100755 diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md old mode 100644 new mode 100755 index e006f302c68..25c804b0035 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -4,7 +4,7 @@ ## distributed_product_mode -Alters the behavior of [distributed subqueries](../../query_language/queries.md#queries-distributed-subrequests), i.e. in cases when the query contains the product of distributed tables. +Changes the behavior of [distributed subqueries](../../query_language/queries.md#queries-distributed-subrequests), i.e. in cases when the query contains the product of distributed tables. ClickHouse applies the configuration if the subqueries on any level have a distributed table that exists on the local server and has more than one shard. @@ -12,9 +12,9 @@ Restrictions: - Only applied for IN and JOIN subqueries. - Used only if a distributed table is used in the FROM clause. -- Not used for a table-valued [ remote](../../table_functions/remote.md#table_functions-remote)function. +- Not used for a table-valued [ remote](../../table_functions/remote.md#table_functions-remote) function. -Possible values: +The possible values ​​are: @@ -36,7 +36,7 @@ Disables query execution if the index can't be used by date. Works with tables in the MergeTree family. -If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree)". +If `force_index_by_date=1`, ClickHouse checks whether the query has a date key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For example, the condition `Date != ' 2000-01-01 '` is acceptable even when it matches all the data in the table (i.e., running the query requires a full scan). For more information about ranges of data in MergeTree tables, see "[MergeTree](../../table_engines/mergetree.md#table_engines-mergetree)". @@ -46,7 +46,7 @@ Disables query execution if indexing by the primary key is not possible. Works with tables in the MergeTree family. -If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about ranges of data in MergeTree tables, see [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree)". +If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../table_engines/mergetree.md#table_engines-mergetree)". @@ -158,7 +158,7 @@ Don't confuse blocks for compression (a chunk of memory consisting of bytes) and ## min_compress_block_size -For [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree) tables. In order to reduce latency when processing queries, a block is compressed when writing the next mark if its size is at least 'min_compress_block_size'. By default, 65,536. +For [MergeTree](../../table_engines/mergetree.md#table_engines-mergetree)" tables. In order to reduce latency when processing queries, a block is compressed when writing the next mark if its size is at least 'min_compress_block_size'. By default, 65,536. The actual size of the block, if the uncompressed data is less than 'max_compress_block_size', is no less than this value and no less than the volume of data for one mark. @@ -253,13 +253,13 @@ Yandex.Metrica uses this parameter set to 1 for implementing suggestions for seg ## schema -This parameter is useful when you are using formats that require a schema definition, such as [ Cap'n Proto](https://capnproto.org/). The value depends on the format. +This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/). The value depends on the format. ## stream_flush_interval_ms -Works for tables with streaming in the case of a timeout, or when a thread generates [ max_insert_block_size](#settings-settings-max_insert_block_size) rows. +Works for tables with streaming in the case of a timeout, or when a thread generates[max_insert_block_size](#settings-settings-max_insert_block_size) rows. The default value is 7500. diff --git a/docs/en/operations/settings/settings_profiles.md b/docs/en/operations/settings/settings_profiles.md old mode 100644 new mode 100755 index c978c599bd5..f1fce41ba75 --- a/docs/en/operations/settings/settings_profiles.md +++ b/docs/en/operations/settings/settings_profiles.md @@ -17,7 +17,7 @@ Example: - + 8 diff --git a/docs/en/operations/tips.md b/docs/en/operations/tips.md old mode 100644 new mode 100755 index 11fc8f6da11..652698fe24c --- a/docs/en/operations/tips.md +++ b/docs/en/operations/tips.md @@ -105,7 +105,7 @@ Use at least a 10 GB network, if possible. 1 Gb will also work, but it will be m You are probably already using ZooKeeper for other purposes. You can use the same installation of ZooKeeper, if it isn't already overloaded. -It's best to use a fresh version of ZooKeeper – 3.5 or later. The version in stable Linux distributions may be outdated. +It's best to use a fresh version of ZooKeeper – 3.4.9 or later. The version in stable Linux distributions may be outdated. With the default settings, ZooKeeper is a time bomb: @@ -174,8 +174,7 @@ dynamicConfigFile=/etc/zookeeper-{{ cluster['name'] }}/conf/zoo.cfg.dynamic Java version: ```text -Java(TM) SE Runtime Environment (build 1.8.0_25-b17) -Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) +Java(TM) SE Runtime Environment (build 1.8.0_25-b17)Java HotSpot(TM) 64-Bit Server VM (build 25.25-b02, mixed mode) ``` JVM parameters: diff --git a/docs/en/operators/index.md b/docs/en/operators/index.md old mode 100644 new mode 100755 index 779f3bf3843..411cde34b50 --- a/docs/en/operators/index.md +++ b/docs/en/operators/index.md @@ -67,11 +67,11 @@ Groups of operators are listed in order of priority (the higher it is in the lis `NOT a` The `not(a) function.` -## Logical 'AND' operator +## Logical AND operator `a AND b` – The`and(a, b) function.` -## Logical 'OR' operator +## Logical OR operator `a OR b` – The `or(a, b) function.` diff --git a/docs/en/query_language/index.md b/docs/en/query_language/index.md old mode 100644 new mode 100755 index 247d76fc6ed..769d94eb4fd --- a/docs/en/query_language/index.md +++ b/docs/en/query_language/index.md @@ -1 +1,2 @@ # Query language + diff --git a/docs/en/query_language/queries.md b/docs/en/query_language/queries.md old mode 100644 new mode 100755 index 743706a551f..d235945a646 --- a/docs/en/query_language/queries.md +++ b/docs/en/query_language/queries.md @@ -11,7 +11,6 @@ CREATE DATABASE [IF NOT EXISTS] db_name `A database` is just a directory for tables. If `IF NOT EXISTS` is included, the query won't return an error if the database already exists. - ## CREATE TABLE @@ -184,7 +183,7 @@ Deletes all tables inside the 'db' database, then deletes the 'db' database itse If `IF EXISTS` is specified, it doesn't return an error if the database doesn't exist. ```sql -DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] +DROP TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` Deletes the table. @@ -308,8 +307,7 @@ SELECT * FROM system.parts WHERE active `active` – Only count active parts. Inactive parts are, for example, source parts remaining after merging to a larger part – these parts are deleted approximately 10 minutes after merging. Another way to view a set of parts and partitions is to go into the directory with table data. -Data directory: `/var/lib/clickhouse/data/database/table/`, -where `/var/lib/clickhouse/` is the path to the ClickHouse data, 'database' is the database name, and 'table' is the table name. Example: +Data directory: `/var/lib/clickhouse/data/database/table/`,where `/var/lib/clickhouse/` is the path to the ClickHouse data, 'database' is the database name, and 'table' is the table name. Example: ```bash $ ls -l /var/lib/clickhouse/data/test/visits/ @@ -325,7 +323,7 @@ Here, `20140317_20140323_2_2_0` and ` 20140317_20140323_4_4_0` are the directori Let's break down the name of the first part: `20140317_20140323_2_2_0`. - `20140317` is the minimum date of the data in the chunk. -- `20140323` is the maximum date of the data in the chunk. +- `20140323` is the maximum data of the data in the chunk. - `2` is the minimum number of the data block. - `2` is the maximum number of the data block. - `0` is the chunk level (the depth of the merge tree it is formed from). @@ -452,7 +450,7 @@ See also the section "Formats". ## SHOW TABLES ```sql -SHOW [TEMPORARY] TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format] +SHOW TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format] ``` Displays a list of tables @@ -462,7 +460,7 @@ Displays a list of tables This query is identical to: `SELECT name FROM system.tables WHERE database = 'db' [AND name LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format]`. -See the section "LIKE operator" also. +See also the section "LIKE operator". ## SHOW PROCESSLIST @@ -486,7 +484,7 @@ Prints a table containing the columns: **query** – The query itself. In INSERT queries, the data for insertion is not output. -**query_id** - The query identifier. Non-empty only if it was explicitly defined by the user. For distributed processing, the query ID is not passed to remote servers. +**query_id** – The query identifier. Non-empty only if it was explicitly defined by the user. For distributed processing, the query ID is not passed to remote servers. This query is identical to: `SELECT * FROM system.processes [INTO OUTFILE filename] [FORMAT format]`. @@ -499,7 +497,7 @@ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'" ## SHOW CREATE TABLE ```sql -SHOW CREATE [TEMPORARY] TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] +SHOW CREATE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` Returns a single `String`-type 'statement' column, which contains a single value – the `CREATE` query used for creating the specified table. @@ -517,7 +515,7 @@ Nested data structures are output in "expanded" format. Each column is shown sep ## EXISTS ```sql -EXISTS [TEMPORARY] TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] +EXISTS TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] ``` Returns a single `UInt8`-type column, which contains the single value `0` if the table or database doesn't exist, or `1` if the table exists in the specified database. @@ -573,9 +571,9 @@ The query can specify a list of columns to insert `[(c1, c2, c3)]`. In this case - The values calculated from the `DEFAULT` expressions specified in the table definition. - Zeros and empty strings, if `DEFAULT` expressions are not defined. -If [strict_insert_defaults=1](../operations/settings/settings.md#settings-strict_insert_defaults), columns that do not have ` DEFAULT` defined must be listed in the query. +If [strict_insert_defaults=1](../operations/settings/settings.md#settings-strict_insert_defaults), columns that do not have `DEFAULT` defined must be listed in the query. -The INSERT can pass data in any [format](../formats/index.md#formats) supported by ClickHouse. The format must be specified explicitly in the query: +Data can be passed to the INSERT in any [format](../formats/index.md#formats) supported by ClickHouse. The format must be specified explicitly in the query: ```sql INSERT INTO [db.]table [(c1, c2, c3)] FORMAT format_name data_set @@ -974,8 +972,7 @@ All columns that are not needed for the JOIN are deleted from the subquery. There are several types of JOINs: -`INNER` or `LEFT` type: -If INNER is specified, the result will contain only those rows that have a matching row in the right table. +`INNER` or `LEFT` type:If INNER is specified, the result will contain only those rows that have a matching row in the right table. If LEFT is specified, any rows in the left table that don't have matching rows in the right table will be assigned the default value - zeros or empty rows. LEFT OUTER may be written instead of LEFT; the word OUTER does not affect anything. `ANY` or `ALL` stringency:If `ANY` is specified and the right table has several matching rows, only the first one found is joined. @@ -1106,7 +1103,7 @@ Example: SELECT domainWithoutWWW(URL) AS domain, count(), - any(Title) AS title -- getting the first occurred page header for each domain. + any(Title) AS title -- getting the first occurring page header for each domain. FROM hits GROUP BY domain ``` @@ -1351,7 +1348,7 @@ There are two options for IN-s with subqueries (similar to JOINs): normal `IN`
-Remember that the algorithms described below may work differently depending on the [](../operations/settings/settings.md#settings-distributed_product_mode) `distributed_product_mode` setting. +Remember that the algorithms described below may work differently depending on the [settings](../operations/settings/settings.md#settings-distributed_product_mode) `distributed_product_mode` setting.
@@ -1479,34 +1476,34 @@ In all other cases, we don't recommend using the asterisk, since it only gives y ## KILL QUERY ```sql -KILL QUERY - WHERE - [SYNC|ASYNC|TEST] - [FORMAT format] +KILL QUERY WHERE [SYNC|ASYNC|TEST] [FORMAT format] ``` Attempts to terminate queries currently running. -The queries to terminate are selected from the system.processes table for which `WHERE` expression is true. +The queries to terminate are selected from the system.processes table for which expression_for_system.processes is true. Examples: ```sql --- Terminates all queries with the specified query_id. KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' +``` --- Synchronously terminates all queries run by `username`. +Terminates all queries with the specified query_id. + +```sql KILL QUERY WHERE user='username' SYNC ``` +Synchronously terminates all queries run by `username`. + Readonly-users can only terminate their own requests. - -By default, the asynchronous version of queries is used (`ASYNC`), which doesn't wait for query termination. - -The synchronous version (`SYNC`) waits for all queries to be killed and displays information about each process as it terminates. +By default, the asynchronous version of queries is used (`ASYNC`), which terminates without waiting for queries to complete. +The synchronous version (`SYNC`) waits for all queries to be completed and displays information about each process as it terminates. The response contains the `kill_status` column, which can take the following values: -1. 'finished' – The query terminated successfully. +1. 'finished' – The query completed successfully. 2. 'waiting' – Waiting for the query to finish after sending it a signal to terminate. 3. The other values ​​explain why the query can't be terminated. A test query (`TEST`) only checks the user's rights and displays a list of queries to terminate. + diff --git a/docs/en/query_language/syntax.md b/docs/en/query_language/syntax.md old mode 100644 new mode 100755 index e151d2ee3d9..4928f2d4a12 --- a/docs/en/query_language/syntax.md +++ b/docs/en/query_language/syntax.md @@ -46,7 +46,7 @@ There are numeric literals, string literals, and compound literals. A numeric literal tries to be parsed: -- first as a 64-bit signed number, using the 'strtoull' function. +- First as a 64-bit signed number, using the 'strtoull' function. - If unsuccessful, as a 64-bit unsigned number, using the 'strtoll' function. - If unsuccessful, as a floating-point number using the 'strtod' function. - Otherwise, an error is returned. diff --git a/docs/en/roadmap.md b/docs/en/roadmap.md old mode 100644 new mode 100755 index 46b08a89607..8241b0a65ae --- a/docs/en/roadmap.md +++ b/docs/en/roadmap.md @@ -3,11 +3,13 @@ ## Q1 2018 ### New functionality -- Initial support for `UPDATE` and `DELETE`. -- Multi-dimensional and nested arrays. - - It may look like this: - + +- Support for `UPDATE` and `DELETE`. + +- Multidimensional and nested arrays. + + It can look something like this: + ```sql CREATE TABLE t ( @@ -21,7 +23,7 @@ ENGINE = MergeTree ORDER BY x - External MySQL and ODBC tables. - External tables can be integrated to ClickHouse using external dictionaries. This will be an alternative and a more convenient way to do so. + External tables can be integrated into ClickHouse using external dictionaries. This new functionality is a convenient alternative to connecting external tables. ```sql SELECT ... @@ -30,66 +32,68 @@ FROM mysql('host:port', 'db', 'table', 'user', 'password')` ### Improvements -- Efficient data copy between ClickHouse clusters. +- Effective data copying between ClickHouse clusters. - Currently, it is possible to copy data using remote() function, e.g.: ` + Now you can copy data with the remote() function. For example: ` INSERT INTO t SELECT * FROM remote(...) `. - The performance of this will be improved by proper distributed execution. + This operation will have improved performance. - O_DIRECT for merges. - Should improve OS cache performance and correspondingly query performance for 'hot' queries. - + This will improve the performance of the OS cache and "hot" queries. ## Q2 2018 ### New functionality -- UPDATE/DELETE in order to comply with Europe GDPR. -- Protobuf and Parquet input/output formats. -- Create dictionaries by DDL queries. +- UPDATE/DELETE conform to the EU GDPR. +- Protobuf and Parquet input and output formats. +- Creating dictionaries using DDL queries. - Currently, it is inconvenient and confusing that dictionaries are defined in external XML files while being a part of DB schema. The new approach will fix that. + Currently, dictionaries that are part of the database schema are defined in external XML files. This is inconvenient and counter-intuitive. The new approach should fix it. + +- Integration with LDAP. -- LDAP integration. - WITH ROLLUP and WITH CUBE for GROUP BY. -- Custom encoding/compression for columns. - Currently, ClickHouse support LZ4 and ZSTD compressions for columns, and compressions settings are global (see our article [Compression in ClickHouse](https://www.altinity.com/blog/2017/11/21/compression-in-clickhouse)) for more details). Column level encoding (e.g. delta encoding) and compression will allow more efficient data storage and therefore faster queries. +- Custom encoding and compression for each column individually. -- Store data at multiple disk volumes of a single server. + As of now, ClickHouse supports LZ4 and ZSTD compression of columns, and compression settings are global (see the article [Compression in ClickHouse](https://www.altinity.com/blog/2017/11/21/compression-in-clickhouse)). Per-column compression and encoding will provide more efficient data storage, which in turn will speed up queries. - That will make it easier to extend disk system as well as use different disk systems for different DBs or tables. Currently, users have to use symlinks if DB/table needs to be stored in another volume. +- Storing data on multiple disks on the same server. + + This functionality will make it easier to extend the disk space, since different disk systems can be used for different databases or tables. Currently, users are forced to use symbolic links if the databases and tables must be stored on a different disk. ### Improvements -A lot of enhancements and fixes are planned for query execution. In particular: +Many improvements and fixes are planned for the query execution system. For example: -- Using index for ‘in (subquery)’. +- Using an index for `in (subquery)`. - Currently, index is not used for such queries resulting in lower performance. + The index is not used right now, which reduces performance. -- Predicate pushdown from ‘where’ into subqueries and Predicate pushdown for views. +- Passing predicates from `where` to subqueries, and passing predicates to views. - These two are related since view is replaced by subquery. Currently, performance of filter conditions for views is significantly degraded, views can not use primary key of the underlying table, that makes views on big tables pretty much useless. + The predicates must be passed, since the view is changed by the subquery. Performance is still low for view filters, and views can't use the primary key of the original table, which makes views useless for large tables. -- Short-circuit expressions evaluation (ternary operator, if, multiIf). +- Optimizing branching operations (ternary operator, if, multiIf). - Currently, ClickHouse evaluates all branches even if the first one needs to be returned due to logical condition result. + ClickHouse currently performs all branches, even if they aren't necessary. -- Using primary key for GROUP BY and ORDER BY. +- Using a primary key for GROUP BY and ORDER BY. - This may speed up certain types of queries since data is already partially pre-sorted. + This will speed up certain types of queries with partially sorted data. ## Q3-Q4 2018 -Longer term plans are not yet finalized. There are two major projects on the list so far. +We don't have any set plans yet, but the main projects will be: -- Resource pools for query execution. +- Resource pools for executing queries. - That will allow managing workloads more efficiently. + This will make load management more efficient. - ANSI SQL JOIN syntax. - That will make ClickHouse more friendly for numerous SQL tools. + Improve ClickHouse compatibility with many SQL tools. + diff --git a/docs/en/system_tables/index.md b/docs/en/system_tables/index.md old mode 100644 new mode 100755 index 614ce4020ec..240105a684b --- a/docs/en/system_tables/index.md +++ b/docs/en/system_tables/index.md @@ -4,5 +4,5 @@ System tables are used for implementing part of the system's functionality, and You can't delete a system table (but you can perform DETACH). System tables don't have files with data on the disk or files with metadata. The server creates all the system tables when it starts. System tables are read-only. -System tables are located in the 'system' database. +They are located in the 'system' database. diff --git a/docs/en/system_tables/system.asynchronous_metrics.md b/docs/en/system_tables/system.asynchronous_metrics.md old mode 100644 new mode 100755 diff --git a/docs/en/system_tables/system.clusters.md b/docs/en/system_tables/system.clusters.md old mode 100644 new mode 100755 index c0bc3dd13fa..bc8dab86b3c --- a/docs/en/system_tables/system.clusters.md +++ b/docs/en/system_tables/system.clusters.md @@ -4,13 +4,12 @@ Contains information about clusters available in the config file and the servers Columns: ```text -cluster String – Cluster name. -shard_num UInt32 – Number of a shard in the cluster, starting from 1. -shard_weight UInt32 – Relative weight of a shard when writing data. -replica_num UInt32 – Number of a replica in the shard, starting from 1. -host_name String – Host name as specified in the config. -host_address String – Host's IP address obtained from DNS. -port UInt16 – The port used to access the server. -user String – The username to use for connecting to the server. +cluster String - Cluster name. +shard_num UInt32 - Number of a shard in the cluster, starting from 1. +shard_weight UInt32 - Relative weight of a shard when writing data. +replica_num UInt32 - Number of a replica in the shard, starting from 1. +host_name String - Host name as specified in the config. +host_address String - Host's IP address obtained from DNS. +port UInt16 - The port used to access the server. +user String - The username to use for connecting to the server. ``` - diff --git a/docs/en/system_tables/system.columns.md b/docs/en/system_tables/system.columns.md old mode 100644 new mode 100755 index 975b84fe9d4..bf05616fbef --- a/docs/en/system_tables/system.columns.md +++ b/docs/en/system_tables/system.columns.md @@ -11,4 +11,3 @@ type String - Column type. default_type String - Expression type (DEFAULT, MATERIALIZED, ALIAS) for the default value, or an empty string if it is not defined. default_expression String - Expression for the default value, or an empty string if it is not defined. ``` - diff --git a/docs/en/system_tables/system.databases.md b/docs/en/system_tables/system.databases.md old mode 100644 new mode 100755 diff --git a/docs/en/system_tables/system.dictionaries.md b/docs/en/system_tables/system.dictionaries.md old mode 100644 new mode 100755 index f3c9929d38e..4ef0d7707b8 --- a/docs/en/system_tables/system.dictionaries.md +++ b/docs/en/system_tables/system.dictionaries.md @@ -5,19 +5,19 @@ Contains information about external dictionaries. Columns: ```text -name String – Dictionary name. -type String – Dictionary type: Flat, Hashed, Cache. -origin String – Path to the config file where the dictionary is described.attribute. -names Array(String) – Array of attribute names provided by the dictionary. -attribute.types Array(String) – Corresponding array of attribute types provided by the dictionary. -has_hierarchy UInt8 – Whether the dictionary is hierarchical. -bytes_allocated UInt64 – The amount of RAM used by the dictionary. -hit_rate Float64 – For cache dictionaries, the percent of usage for which the value was in the cache. -element_count UInt64 – The number of items stored in the dictionary. -load_factor Float64 – The filled percentage of the dictionary (for a hashed dictionary, it is the filled percentage of the hash table). -creation_time DateTime – Time spent for the creation or last successful reload of the dictionary. -last_exception String – Text of an error that occurred when creating or reloading the dictionary, if the dictionary couldn't be created. -source String – Text describing the data source for the dictionary. +name String - Dictionary name. +type String - Dictionary type: Flat, Hashed, Cache. +origin String - Path to the config file where the dictionary is described. +attribute.names Array(String) - Array of attribute names provided by the dictionary. +attribute.types Array(String) - Corresponding array of attribute types provided by the dictionary. +has_hierarchy UInt8 - Whether the dictionary is hierarchical. +bytes_allocated UInt64 - The amount of RAM used by the dictionary. +hit_rate Float64 - For cache dictionaries, the percent of usage for which the value was in the cache. +element_count UInt64 - The number of items stored in the dictionary. +load_factor Float64 - The filled percentage of the dictionary (for a hashed dictionary, it is the filled percentage of the hash table). +creation_time DateTime - Time spent for the creation or last successful reload of the dictionary. +last_exception String - Text of an error that occurred when creating or reloading the dictionary, if the dictionary couldn't be created. +source String - Text describing the data source for the dictionary. ``` Note that the amount of memory used by the dictionary is not proportional to the number of items stored in it. So for flat and cached dictionaries, all the memory cells are pre-assigned, regardless of how full the dictionary actually is. diff --git a/docs/en/system_tables/system.events.md b/docs/en/system_tables/system.events.md old mode 100644 new mode 100755 diff --git a/docs/en/system_tables/system.functions.md b/docs/en/system_tables/system.functions.md old mode 100644 new mode 100755 index ac550acc14b..a1022a5e557 --- a/docs/en/system_tables/system.functions.md +++ b/docs/en/system_tables/system.functions.md @@ -6,6 +6,6 @@ Columns: ```text name String – Function name. -is_aggregate UInt8 – Whether it is an aggregate function. +is_aggregate UInt8 – Whether it is an aggregate function. ``` diff --git a/docs/en/system_tables/system.merges.md b/docs/en/system_tables/system.merges.md old mode 100644 new mode 100755 index 0a10e4a5a8c..59870922ea5 --- a/docs/en/system_tables/system.merges.md +++ b/docs/en/system_tables/system.merges.md @@ -18,4 +18,3 @@ rows_read UInt64 - Number of rows read. bytes_written_uncompressed UInt64 - Amount of bytes written, uncompressed. rows_written UInt64 - Number of rows written. ``` - diff --git a/docs/en/system_tables/system.numbers.md b/docs/en/system_tables/system.numbers.md old mode 100644 new mode 100755 diff --git a/docs/en/system_tables/system.numbers_mt.md b/docs/en/system_tables/system.numbers_mt.md old mode 100644 new mode 100755 diff --git a/docs/en/system_tables/system.one.md b/docs/en/system_tables/system.one.md old mode 100644 new mode 100755 diff --git a/docs/en/system_tables/system.parts.md b/docs/en/system_tables/system.parts.md old mode 100644 new mode 100755 diff --git a/docs/en/system_tables/system.processes.md b/docs/en/system_tables/system.processes.md old mode 100644 new mode 100755 index b9ad6a44e81..0802e555648 --- a/docs/en/system_tables/system.processes.md +++ b/docs/en/system_tables/system.processes.md @@ -14,12 +14,12 @@ rows_read UInt64 – The number of rows read from the table. For distribu bytes_read UInt64 – The number of uncompressed bytes read from the table. For distributed processing, on the requestor server, this is the total for all remote servers. -total_rows_approx UInt64 – The approximate total number of rows that must be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. +UInt64 total_rows_approx – The approximate total number of rows that must be read. For distributed processing, on the requestor server, this is the total for all remote servers. It can be updated during request processing, when new sources to process become known. memory_usage UInt64 – Memory consumption by the query. It might not include some types of dedicated memory. -query String – The query text. For INSERT, it doesn't include the data to insert. +Query String – The query text. For INSERT, it doesn't include the data to insert. -query_id String - The query ID, if defined. +query_id – Query ID, if defined. ``` diff --git a/docs/en/system_tables/system.replicas.md b/docs/en/system_tables/system.replicas.md old mode 100644 new mode 100755 index ec1341198dc..75cd8e34340 --- a/docs/en/system_tables/system.replicas.md +++ b/docs/en/system_tables/system.replicas.md @@ -42,37 +42,41 @@ Columns: database: database name table: table name engine: table engine name + is_leader: whether the replica is the leader Only one replica at a time can be the leader. The leader is responsible for selecting background merges to perform. Note that writes can be performed to any replica that is available and has a session in ZK, regardless of whether it is a leader. -is_readonly: Whether the replica is in read-only mode.This mode is turned on if the config doesn't have sections with ZK, if an unknown error occurred when reinitializing sessions in ZK, and during session reinitialization in ZK. +is_readonly: Whether the replica is in read-only mode. +This mode is turned on if the config doesn't have sections with ZK, if an unknown error occurred when reinitializing sessions in ZK, and during session reinitialization in ZK. is_session_expired: Whether the ZK session expired. Basically, the same thing as is_readonly. -future_parts: The number of data parts that will appear as the result of INSERTs or merges that haven't been done yet. +future_parts: The number of data parts that will appear as the result of INSERTs or merges that haven't been done yet. -parts_to_check: The number of data parts in the queue for verification. A part is put in the verification queue if there is suspicion that it might be damaged. +parts_to_check: The number of data parts in the queue for verification. +A part is put in the verification queue if there is suspicion that it might be damaged. -zookeeper_path: The path to the table data in ZK. +zookeeper_path: The path to the table data in ZK. replica_name: Name of the replica in ZK. Different replicas of the same table have different names. -replica_path: The path to the replica data in ZK. The same as concatenating zookeeper_path/replicas/replica_path. +replica_path: The path to the replica data in ZK. The same as concatenating zookeeper_path/replicas/replica_path. -columns_version: Version number of the table structure. Indicates how many times ALTER was performed. If replicas have different versions, it means some replicas haven't made all of the ALTERs yet. +columns_version: Version number of the table structure. Indicates how many times ALTER was performed. If replicas have different versions, it means some replicas haven't made all of the ALTERs yet. -queue_size: Size of the queue for operations waiting to be performed. Operations include inserting blocks of data, merges, and certain other actions. +queue_size: Size of the queue for operations waiting to be performed. +Operations include inserting blocks of data, merges, and certain other actions. Normally coincides with future_parts. -inserts_in_queue: Number of inserts of blocks of data that need to be made. Insertions are usually replicated fairly quickly. If the number is high, something is wrong. +inserts_in_queue: Number of inserts of blocks of data that need to be made. Insertions are usually replicated fairly quickly. If the number is high, something is wrong. -merges_in_queue: The number of merges waiting to be made. Sometimes merges are lengthy, so this value may be greater than zero for a long time. +merges_in_queue: The number of merges waiting to be made. Sometimes merges are lengthy, so this value may be greater than zero for a long time. The next 4 columns have a non-null value only if the ZK session is active. -log_max_index: Maximum entry number in the log of general activity. -log_pointer: Maximum entry number in the log of general activity that the replica copied to its execution queue, plus one. If log_pointer is much smaller than log_max_index, something is wrong. +log_max_index: Maximum entry number in the log of general activity. log_pointer: Maximum entry number in the log of general activity that the replica copied to its execution queue, plus one. +If log_pointer is much smaller than log_max_index, something is wrong. total_replicas: Total number of known replicas of this table. active_replicas: Number of replicas of this table that have a ZK session (the number of active replicas). diff --git a/docs/en/system_tables/system.settings.md b/docs/en/system_tables/system.settings.md old mode 100644 new mode 100755 index a055135ebcf..90a392bcc24 --- a/docs/en/system_tables/system.settings.md +++ b/docs/en/system_tables/system.settings.md @@ -6,9 +6,9 @@ I.e. used for executing the query you are using to read from the system.settings Columns: ```text -name String – Setting name. +name String – Setting name. value String – Setting value. -changed UInt8 -–Whether the setting was explicitly defined in the config or explicitly changed. +changed UInt8 - Whether the setting was explicitly defined in the config or explicitly changed. ``` Example: diff --git a/docs/en/system_tables/system.tables.md b/docs/en/system_tables/system.tables.md old mode 100644 new mode 100755 index fabddf4dbb1..5757a8ac3da --- a/docs/en/system_tables/system.tables.md +++ b/docs/en/system_tables/system.tables.md @@ -1,6 +1,7 @@ # system.tables This table contains the String columns 'database', 'name', and 'engine'. -Also, the table has three virtual columns: metadata_modification_time of type DateTime, create_table_query and engine_full of type String. +The table also contains three virtual columns: metadata_modification_time (DateTime type), create_table_query, and engine_full (String type). Each table that the server knows about is entered in the 'system.tables' table. This system table is used for implementing SHOW TABLES queries. + diff --git a/docs/en/system_tables/system.zookeeper.md b/docs/en/system_tables/system.zookeeper.md old mode 100644 new mode 100755 index c456e7d9207..46b40e7a08f --- a/docs/en/system_tables/system.zookeeper.md +++ b/docs/en/system_tables/system.zookeeper.md @@ -70,4 +70,3 @@ numChildren: 7 pzxid: 987021252247 path: /clickhouse/tables/01-08/visits/replicas ``` - diff --git a/docs/en/table_engines/aggregatingmergetree.md b/docs/en/table_engines/aggregatingmergetree.md old mode 100644 new mode 100755 index d75d8353e6d..987c102508b --- a/docs/en/table_engines/aggregatingmergetree.md +++ b/docs/en/table_engines/aggregatingmergetree.md @@ -1,10 +1,11 @@ # AggregatingMergeTree -This engine differs from MergeTree in that the merge combines the states of aggregate functions stored in the table for rows with the same primary key value. +This engine differs from `MergeTree` in that the merge combines the states of aggregate functions stored in the table for rows with the same primary key value. -In order for this to work, it uses the AggregateFunction data type and the -State and -Merge modifiers for aggregate functions. Let's examine it more closely. +For this to work, it uses the `AggregateFunction` data type, as well as `-State` and `-Merge` modifiers for aggregate functions. Let's examine it more closely. + +There is an `AggregateFunction` data type. It is a parametric data type. As parameters, the name of the aggregate function is passed, then the types of its arguments. -There is an AggregateFunction data type. It is a parametric data type. As parameters, the name of the aggregate function is passed, then the types of its arguments. Examples: ```sql @@ -19,12 +20,16 @@ CREATE TABLE t This type of column stores the state of an aggregate function. To get this type of value, use aggregate functions with the `State` suffix. -Example: `uniqState(UserID), quantilesState(0.5, 0.9)(SendTiming)` – in contrast to the corresponding 'uniq' and 'quantiles' functions, these functions return the state, rather than the prepared value. In other words, they return an AggregateFunction type value. -An AggregateFunction type value can't be output in Pretty formats. In other formats, these types of values are output as implementation-specific binary data. The AggregateFunction type values are not intended for output or saving in a dump. +Example: +`uniqState(UserID), quantilesState(0.5, 0.9)(SendTiming)` -The only useful thing you can do with AggregateFunction type values is combine the states and get a result, which essentially means to finish aggregation. Aggregate functions with the 'Merge' suffix are used for this purpose. -Example: `uniqMerge(UserIDState), where UserIDState has the AggregateFunction type`. +In contrast to the corresponding `uniq` and `quantiles` functions, these functions return the state, rather than the prepared value. In other words, they return an `AggregateFunction` type value. + +An `AggregateFunction` type value can't be output in Pretty formats. In other formats, these types of values are output as implementation-specific binary data. The `AggregateFunction` type values are not intended for output or saving in a dump. + +The only useful thing you can do with `AggregateFunction` type values is combine the states and get a result, which essentially means to finish aggregation. Aggregate functions with the 'Merge' suffix are used for this purpose. +Example: `uniqMerge(UserIDState), where UserIDState has the AggregateFunction` type. In other words, an aggregate function with the 'Merge' suffix takes a set of states, combines them, and returns the result. As an example, these two queries return the same result: @@ -37,15 +42,15 @@ SELECT uniqMerge(state) FROM (SELECT uniqState(UserID) AS state FROM table GROUP There is an ` AggregatingMergeTree` engine. Its job during a merge is to combine the states of aggregate functions from different table rows with the same primary key value. -You can't use a normal INSERT to insert a row in a table containing AggregateFunction columns, because you can't explicitly define the AggregateFunction value. Instead, use INSERT SELECT with '-State' aggregate functions for inserting data. +You can't use a normal INSERT to insert a row in a table containing `AggregateFunction` columns, because you can't explicitly define the `AggregateFunction` value. Instead, use `INSERT SELECT` with `-State` aggregate functions for inserting data. -With SELECT from an AggregatingMergeTree table, use GROUP BY and aggregate functions with the '-Merge' modifier in order to complete data aggregation. +With SELECT from an `AggregatingMergeTree` table, use GROUP BY and aggregate functions with the '-Merge' modifier in order to complete data aggregation. -You can use AggregatingMergeTree tables for incremental data aggregation, including for aggregated materialized views. +You can use `AggregatingMergeTree` tables for incremental data aggregation, including for aggregated materialized views. Example: -Creating a materialized AggregatingMergeTree view that tracks the 'test.visits' table: +Create an `AggregatingMergeTree` materialized view that watches the `test.visits` table: ```sql CREATE MATERIALIZED VIEW test.basic @@ -59,13 +64,13 @@ FROM test.visits GROUP BY CounterID, StartDate; ``` -Inserting data in the 'test.visits' table. Data will also be inserted in the view, where it will be aggregated: +Insert data in the `test.visits` table. Data will also be inserted in the view, where it will be aggregated: ```sql INSERT INTO test.visits ... ``` -Performing SELECT from the view using GROUP BY to finish data aggregation: +Perform `SELECT` from the view using `GROUP BY` in order to complete data aggregation: ```sql SELECT diff --git a/docs/en/table_engines/buffer.md b/docs/en/table_engines/buffer.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/collapsingmergetree.md b/docs/en/table_engines/collapsingmergetree.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/custom_partitioning_key.md b/docs/en/table_engines/custom_partitioning_key.md old mode 100644 new mode 100755 index 6a468a1137d..bfcb3c2c545 --- a/docs/en/table_engines/custom_partitioning_key.md +++ b/docs/en/table_engines/custom_partitioning_key.md @@ -2,7 +2,7 @@ # Custom partitioning key -Starting with version 1.1.54310, you can create tables in the MergeTree family with any partition expression (not only partitioning by month). +Starting with version 1.1.54310, you can create tables in the MergeTree family with any partitioning expression (not only partitioning by month). The partition key can be an expression from the table columns, or a tuple of such expressions (similar to the primary key). The partition key can be omitted. When creating a table, specify the partition key in the ENGINE description with the new syntax: @@ -10,7 +10,7 @@ The partition key can be an expression from the table columns, or a tuple of suc ENGINE [=] Name(...) [PARTITION BY expr] [ORDER BY expr] [SAMPLE BY expr] [SETTINGS name=value, ...] ``` -For MergeTree tables, the partition expression is specified after `PARTITION BY`, the primary key after `ORDER BY`, the sampling key after `SAMPLE BY`, and `SETTINGS` can specify `index_granularity` (optional; the default value is 8192), as well as other settings from [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). Example: +For MergeTree tables, the partition expression is specified after `PARTITION BY`, the primary key after `ORDER BY`, the sampling key after `SAMPLE BY`, and `SETTINGS` can specify `index_granularity` (optional; the default value is 8192), as well as other settings from [MergeTreeSettings.h](https://github.com/yandex/ClickHouse/blob/master/dbms/src/Storages/MergeTree/MergeTreeSettings.h). The other engine parameters are specified in parentheses after the engine name, as previously. Example: ```sql ENGINE = ReplicatedCollapsingMergeTree('/clickhouse/tables/name', 'replica1', Sign) @@ -23,7 +23,7 @@ The traditional partitioning by month is expressed as `toYYYYMM(date_column)`. You can't convert an old-style table to a table with custom partitions (only via INSERT SELECT). -After this table is created, merge will only work for data parts that have the same value for the partition expression. Note: This means that you shouldn't make overly granular partitions (more than about a thousand partitions), or SELECT will perform poorly. +After this table is created, merge will only work for data parts that have the same value for the partitioning expression. Note: This means that you shouldn't make overly granular partitions (more than about a thousand partitions), or SELECT will perform poorly. To specify a partition in ALTER PARTITION commands, specify the value of the partition expression (or a tuple). Constants and constant expressions are supported. Example: @@ -35,13 +35,13 @@ Deletes the partition for the current week with event type 1. The same is true f Note: For old-style tables, the partition can be specified either as a number `201710` or a string `'201710'`. The syntax for the new style of tables is stricter with types (similar to the parser for the VALUES input format). In addition, ALTER TABLE FREEZE PARTITION uses exact match for new-style tables (not prefix match). -In the `system.parts` table, the `partition` column should specify the value of the partition expression to use in ALTER queries (if quotas are removed). The `name` column should specify the name of the data part that has a new format. +In the `system.parts` table, the `partition` column specifies the value of the partition expression to use in ALTER queries (if quotas are removed). The `name` column should specify the name of the data part that has a new format. -Before: `20140317_20140323_2_2_0` (minimal data - maximal data - number of minimal block - number of maximal block - level). +Was: `20140317_20140323_2_2_0` (minimum date - maximum date - minimum block number - maximum block number - level). -After: `201403_2_2_0` (partition ID - number of minimal block - number of maximal block - level). +Now: `201403_2_2_0` (partition ID - minimum block number - maximum block number - level). The partition ID is its string identifier (human-readable, if possible) that is used for the names of data parts in the file system and in ZooKeeper. You can specify it in ALTER queries in place of the partition key. Example: Partition key `toYYYYMM(EventDate)`; ALTER can specify either `PARTITION 201710` or `PARTITION ID '201710'`. -There are more examples in the tests [`00502_custom_partitioning_local`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). +For more examples, see the tests [`00502_custom_partitioning_local`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_local.sql) and [`00502_custom_partitioning_replicated_zookeeper`](https://github.com/yandex/ClickHouse/blob/master/dbms/tests/queries/0_stateless/00502_custom_partitioning_replicated_zookeeper.sql). diff --git a/docs/en/table_engines/dictionary.md b/docs/en/table_engines/dictionary.md new file mode 100755 index 00000000000..ae8cca90d7c --- /dev/null +++ b/docs/en/table_engines/dictionary.md @@ -0,0 +1,106 @@ + + +# Dictionary + +The `Dictionary` engine displays the dictionary data as a ClickHouse table. + +As an example, consider a dictionary of `products` with the following configuration: + +```xml + + + products + + + products
+ DSN=some-db-server +
+ + + 300 + 360 + + + + + + + product_id + + + title + String + + + +
+
+``` + +Query the dictionary data: + +```sql +select name, type, key, attribute.names, attribute.types, bytes_allocated, element_count,source from system.dictionaries where name = 'products'; + +SELECT + name, + type, + key, + attribute.names, + attribute.types, + bytes_allocated, + element_count, + source +FROM system.dictionaries +WHERE name = 'products' +``` +``` +┌─name─────┬─type─┬─key────┬─attribute.names─┬─attribute.types─┬─bytes_allocated─┬─element_count─┬─source──────────┐ +│ products │ Flat │ UInt64 │ ['title'] │ ['String'] │ 23065376 │ 175032 │ ODBC: .products │ +└──────────┴──────┴────────┴─────────────────┴─────────────────┴─────────────────┴───────────────┴─────────────────┘ +``` + +You can use the [dictGet*](../functions/ext_dict_functions.md#ext_dict_functions) function to get the dictionary data in this format. + +This view isn't helpful when you need to get raw data, or when performing a `JOIN` operation. For these cases, you can use the `Dictionary` engine, which displays the dictionary data in a table. + +Syntax: + +``` +CREATE TABLE %table_name% (%fields%) engine = Dictionary(%dictionary_name%)` +``` + +Usage example: + +```sql +create table products (product_id UInt64, title String) Engine = Dictionary(products); + +CREATE TABLE products +( + product_id UInt64, + title String, +) +ENGINE = Dictionary(products) +``` +``` +Ok. + +0 rows in set. Elapsed: 0.004 sec. +``` + +Take a look at what's in the table. + +```sql +select * from products limit 1; + +SELECT * +FROM products +LIMIT 1 +``` +``` +┌────product_id─┬─title───────────┐ +│ 152689 │ Некоторый товар │ +└───────────────┴─────────────────┘ + +1 rows in set. Elapsed: 0.006 sec. +``` diff --git a/docs/en/table_engines/distributed.md b/docs/en/table_engines/distributed.md old mode 100644 new mode 100755 index b8643461fbb..dd2ffe27fe5 --- a/docs/en/table_engines/distributed.md +++ b/docs/en/table_engines/distributed.md @@ -25,29 +25,29 @@ Clusters are set like this: - - 1 - - false - - example01-01-1 - 9000 - - - example01-01-2 - 9000 - - - - 2 - false - - example01-02-1 - 9000 - - - example01-02-2 - 9000 + + 1 + + false + + example01-01-1 + 9000 + + + example01-01-2 + 9000 + + + + 2 + false + + example01-02-1 + 9000 + + + example01-02-2 + 9000 diff --git a/docs/en/table_engines/external_data.md b/docs/en/table_engines/external_data.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/file.md b/docs/en/table_engines/file.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/graphitemergetree.md b/docs/en/table_engines/graphitemergetree.md old mode 100644 new mode 100755 index 6452377ac15..a4b62424954 --- a/docs/en/table_engines/graphitemergetree.md +++ b/docs/en/table_engines/graphitemergetree.md @@ -2,13 +2,13 @@ # GraphiteMergeTree -This engine is designed for rollup (thinning and aggregating/averaging) [ Graphite](http://graphite.readthedocs.io/en/latest/index.html) data. It may be helpful to developers who want to use ClickHouse as a data store for Graphite. +This engine is designed for rollup (thinning and aggregating/averaging) [Graphite](http://graphite.readthedocs.io/en/latest/index.html) data. It may be helpful to developers who want to use ClickHouse as a data store for Graphite. Graphite stores full data in ClickHouse, and data can be retrieved in the following ways: - Without thinning. - Using the [MergeTree](mergetree.md#table_engines-mergetree) engine. + Uses the [MergeTree](mergetree.md#table_engines-mergetree) engine. - With thinning. @@ -83,4 +83,3 @@ Example of settings:
``` - diff --git a/docs/en/table_engines/index.md b/docs/en/table_engines/index.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/join.md b/docs/en/table_engines/join.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/kafka.md b/docs/en/table_engines/kafka.md old mode 100644 new mode 100755 index 85943e44fc5..4f10e55d029 --- a/docs/en/table_engines/kafka.md +++ b/docs/en/table_engines/kafka.md @@ -59,7 +59,7 @@ Example: level String, total UInt64 ) ENGINE = SummingMergeTree(day, (day, level), 8192); - + CREATE MATERIALIZED VIEW consumer TO daily AS SELECT toDate(toDateTime(timestamp)) AS day, level, count() as total FROM queue GROUP BY day, level; diff --git a/docs/en/table_engines/log.md b/docs/en/table_engines/log.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/materializedview.md b/docs/en/table_engines/materializedview.md old mode 100644 new mode 100755 index 00f70bd72bd..5e2741c6aa1 --- a/docs/en/table_engines/materializedview.md +++ b/docs/en/table_engines/materializedview.md @@ -1,4 +1,4 @@ # MaterializedView -Used for implementing materialized views (for more information, see [CREATE TABLE](../query_language/queries.md#query_language-queries-create_table)). For storing data, it uses a different engine that was specified when creating the view. When reading from a table, it just uses this engine. +Used for implementing materialized views (for more information, see the [CREATE TABLE](../query_language/queries.md#query_language-queries-create_table)) query. For storing data, it uses a different engine that was specified when creating the view. When reading from a table, it just uses this engine. diff --git a/docs/en/table_engines/memory.md b/docs/en/table_engines/memory.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/merge.md b/docs/en/table_engines/merge.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/mergetree.md b/docs/en/table_engines/mergetree.md old mode 100644 new mode 100755 index 71197f21b34..fea02e01d72 --- a/docs/en/table_engines/mergetree.md +++ b/docs/en/table_engines/mergetree.md @@ -56,7 +56,7 @@ In this example, the index can't be used: SELECT count() FROM table WHERE CounterID = 34 OR URL LIKE '%upyachka%' ``` -To check whether ClickHouse can use the index when executing the query, use the settings [ force_index_by_date](../operations/settings/settings.md#settings-settings-force_index_by_date) and [ force_primary_key](../operations/settings/settings.md#settings-settings-force_primary_key). +To check whether ClickHouse can use the index when executing the query, use the settings [force_index_by_date](../operations/settings/settings.md#settings-settings-force_index_by_date)and[force_primary_key](../operations/settings/settings.md#settings-settings-force_primary_key). The index by date only allows reading those parts that contain dates from the desired range. However, a data part may contain data for many dates (up to an entire month), while within a single part the data is ordered by the primary key, which might not contain the date as the first column. Because of this, using a query with only a date condition that does not specify the primary key prefix will cause more data to be read than for a single date. diff --git a/docs/en/table_engines/null.md b/docs/en/table_engines/null.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/replacingmergetree.md b/docs/en/table_engines/replacingmergetree.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/replication.md b/docs/en/table_engines/replication.md old mode 100644 new mode 100755 index 1e58878c34e..20dd17e444f --- a/docs/en/table_engines/replication.md +++ b/docs/en/table_engines/replication.md @@ -46,7 +46,7 @@ You can specify any existing ZooKeeper cluster and the system will use a directo If ZooKeeper isn't set in the config file, you can't create replicated tables, and any existing replicated tables will be read-only. -ZooKeeper isn't used for SELECT queries. In other words, replication doesn't affect the productivity of SELECT queries – they work just as fast as for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../operations/settings/settings.md#settings_settings_max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../operations/settings/settings.md#settings-settings-fallback_to_stale_replicas_for_distributed_queries). +ZooKeeper isn't used for SELECT queries. In other words, replication doesn't affect the productivity of SELECT queries – they work just as fast as for non-replicated tables. When querying distributed replicated tables, ClickHouse behavior is controlled by the settings [max_replica_delay_for_distributed_queries](../operations/settings/settings.md#settings_settings_max_replica_delay_for_distributed_queries) and [fallback_to_stale_replicas_for_distributed_queries](../operations/settings/settings.md#settings-settings-fallback_to_stale_replicas_for_distributed_queries). For each INSERT query (more precisely, for each inserted block of data; the INSERT query contains a single block, or per block for every max_insert_block_size = 1048576 rows), approximately ten entries are made in ZooKeeper in several transactions. This leads to slightly longer latencies for INSERT compared to non-replicated tables. But if you follow the recommendations to insert data in batches of no more than one INSERT per second, it doesn't create any problems. The entire ClickHouse cluster used for coordinating one ZooKeeper cluster has a total of several hundred INSERTs per second. The throughput on data inserts (the number of rows per second) is just as high as for non-replicated data. diff --git a/docs/en/table_engines/set.md b/docs/en/table_engines/set.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/summingmergetree.md b/docs/en/table_engines/summingmergetree.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/tinylog.md b/docs/en/table_engines/tinylog.md old mode 100644 new mode 100755 diff --git a/docs/en/table_engines/view.md b/docs/en/table_engines/view.md old mode 100644 new mode 100755 diff --git a/docs/en/table_functions/index.md b/docs/en/table_functions/index.md old mode 100644 new mode 100755 diff --git a/docs/en/table_functions/merge.md b/docs/en/table_functions/merge.md old mode 100644 new mode 100755 diff --git a/docs/en/table_functions/remote.md b/docs/en/table_functions/remote.md old mode 100644 new mode 100755 index 99b0c7bb116..e26e245207b --- a/docs/en/table_functions/remote.md +++ b/docs/en/table_functions/remote.md @@ -52,7 +52,7 @@ example01-{01..02}-1 If you have multiple pairs of curly brackets, it generates the direct product of the corresponding sets. -Addresses and parts of addresses in curly brackets can be separated by the pipe symbol (|). In this case, the corresponding sets of addresses are interpreted as replicas, and the query will be sent to the first healthy replica. The replicas are evaluated in the order currently set in the [load_balancing](../operations/settings/settings.md#settings-load_balancing) setting. +Addresses and parts of addresses in curly brackets can be separated by the pipe symbol (|). In this case, the corresponding sets of addresses are interpreted as replicas, and the query will be sent to the first healthy replica. However, the replicas are iterated in the order currently set in the [load_balancing](../operations/settings/settings.md#settings-load_balancing) setting. Example: diff --git a/docs/en/utils/clickhouse-copier.md b/docs/en/utils/clickhouse-copier.md old mode 100644 new mode 100755 index 25d22f19222..9d15053fe06 --- a/docs/en/utils/clickhouse-copier.md +++ b/docs/en/utils/clickhouse-copier.md @@ -1,40 +1,54 @@ -# clickhouse-copier util + -The util copies tables data from one cluster to new tables of other (possibly the same) cluster in distributed and fault-tolerant manner. +# clickhouse-copier -Configuration of copying tasks is set in special ZooKeeper node (called the `/description` node). -A ZooKeeper path to the description node is specified via `--task-path
` parameter. -So, node `/task/path/description` should contain special XML content describing copying tasks. +Copies data from the tables in one cluster to tables in another (or the same) cluster. -Simultaneously many `clickhouse-copier` processes located on any servers could execute the same task. -ZooKeeper node `/task/path/` is used by the processes to coordinate their work. -You must not add additional child nodes to `/task/path/`. +You can run multiple `clickhouse-copier` instances on different servers to perform the same job. ZooKeeper is used for syncing the processes. -Currently you are responsible for manual launching of all `cluster-copier` processes. -You can launch as many processes as you want, whenever and wherever you want. -Each process try to select the nearest available shard of source cluster and copy some part of data (partition) from it to the whole -destination cluster (with resharding). -Therefore it makes sense to launch cluster-copier processes on the source cluster nodes to reduce the network usage. +After starting, `clickhouse-copier`: -Since the workers coordinate their work via ZooKeeper, in addition to `--task-path
` you have to specify ZooKeeper -cluster configuration via `--config-file ` parameter. Example of `zookeeper.xml`: +- Connects to ZooKeeper and receives: + - Copying jobs. + - The state of the copying jobs. + +- It performs the jobs. + + Each running process chooses the "closest" shard of the source cluster and copies the data into the destination cluster, resharding the data if necessary. + +`clickhouse-copier` tracks the changes in ZooKeeper and applies them on the fly. + +To reduce network traffic, we recommend running `clickhouse-copier` on the same server where the source data is located. + +## Running clickhouse-copier + +The utility should be run manually: + +```bash +clickhouse-copier copier --daemon --config zookeeper.xml --task-path /task/path --base-dir /path/to/dir +``` + +Parameters: + +- `daemon` — Starts `clickhouse-copier` in daemon mode. +- `config` — The path to the `zookeeper.xml` file with the parameters for the connection to ZooKeeper. +- `task-path` — The path to the ZooKeeper node. This node is used for syncing `clickhouse-copier` processes and storing tasks. Tasks are stored in `$task-path/description`. +- `base-dir` — The path to logs and auxiliary files. When it starts, `clickhouse-copier` creates `clickhouse-copier_YYYYMMHHSS_` subdirectories in `$base-dir`. If this parameter is omitted, the directories are created in the directory where `clickhouse-copier` was launched. + +## Format of zookeeper.xml ```xml - + 127.0.0.1 2181 - + ``` -When you run `clickhouse-copier --config-file --task-path ` the process connects to ZooKeeper cluster, reads tasks config from `/task/path/description` and executes them. - -## Format of task config - -Here is an example of `/task/path/description` content: +## Configuration of copying tasks ```xml @@ -69,62 +83,62 @@ Here is an example of `/task/path/description` content: 0 - 3 - + 1 - - + - + source_cluster test hits - + destination_cluster test hits2 - - ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hits2', '{replica}') + + ENGINE=ReplicatedMergeTree('/clickhouse/tables/{cluster}/{shard}/hits2', '{replica}') PARTITION BY toMonday(date) ORDER BY (CounterID, EventDate) - + jumpConsistentHash(intHash64(UserID), 2) - + CounterID != 0 - '2018-02-26' @@ -133,7 +147,7 @@ Here is an example of `/task/path/description` content: - + ... @@ -142,15 +156,5 @@ Here is an example of `/task/path/description` content: ``` -cluster-copier processes watch for `/task/path/description` node update. -So, if you modify the config settings or `max_workers` params, they will be updated. +`clickhouse-copier` tracks the changes in `/task/path/description` and applies them on the fly. For instance, if you change the value of `max_workers`, the number of processes running tasks will also change. -## Example - -```bash -clickhouse-copier copier --daemon --config /path/to/copier/zookeeper.xml --task-path /clickhouse-copier/cluster1_tables_hits --base-dir /path/to/copier_logs -``` - -`--base-dir /path/to/copier_logs` specifies where auxilary and log files of the copier process will be saved. -In this case it will create `/path/to/copier_logs/clickhouse-copier_YYYYMMHHSS_/` dir with log and status-files. -If it is not specified it will use current dir (`/clickhouse-copier_YYYYMMHHSS_/` if it is run as a `--daemon`). diff --git a/docs/en/utils/clickhouse-local.md b/docs/en/utils/clickhouse-local.md old mode 100644 new mode 100755 index d18cc200320..d5fba56271f --- a/docs/en/utils/clickhouse-local.md +++ b/docs/en/utils/clickhouse-local.md @@ -1,4 +1,6 @@ -# The clickhouse-local program + -The `clickhouse-local` program enables you to perform fast processing on local files that store tables, without having to deploy and configure clickhouse-server. +#clickhouse-local + +The `clickhouse-local` program enables you to perform fast processing on local files that store tables, without having to deploy and configure the ClickHouse server. diff --git a/docs/en/utils/index.md b/docs/en/utils/index.md old mode 100644 new mode 100755 index 7a8c5ee5138..cf541cda895 --- a/docs/en/utils/index.md +++ b/docs/en/utils/index.md @@ -1,6 +1,5 @@ -# ClickHouse utilites +# ClickHouse utility -There are several ClickHouse utilites that are separate executable files: +* [clickhouse-local](clickhouse-local.md#utils-clickhouse-local) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. +* [clickhouse-copier](clickhouse-copier.md#utils-clickhouse-copier) — Copies (and reshards) data from one cluster to another cluster. -* `clickhouse-local` allows to execute SQL queries on a local data like `awk` -* `clickhouse-copier` copies (and reshards) immutable data from one cluster to another in a fault-tolerant manner. diff --git a/docs/mkdocs_en.yml b/docs/mkdocs_en.yml index d5fadb3f1e1..012d498f3e2 100644 --- a/docs/mkdocs_en.yml +++ b/docs/mkdocs_en.yml @@ -86,6 +86,7 @@ pages: - 'GraphiteMergeTree': 'table_engines/graphitemergetree.md' - 'Data replication': 'table_engines/replication.md' - 'Distributed': 'table_engines/distributed.md' + - 'Dictionary': 'table_engines/dictionary.md' - 'Merge': 'table_engines/merge.md' - 'Buffer': 'table_engines/buffer.md' - 'File': 'table_engines/file.md' @@ -95,6 +96,7 @@ pages: - 'View': 'table_engines/view.md' - 'MaterializedView': 'table_engines/materializedview.md' - 'Kafka': 'table_engines/kafka.md' + - 'MySQL': 'table_engines/mysql.md' - 'External data for query processing': 'table_engines/external_data.md' - 'System tables': diff --git a/docs/ru/dicts/external_dicts_dict_layout.md b/docs/ru/dicts/external_dicts_dict_layout.md index ff1b9c0cdd5..defb0605c0f 100644 --- a/docs/ru/dicts/external_dicts_dict_layout.md +++ b/docs/ru/dicts/external_dicts_dict_layout.md @@ -109,15 +109,15 @@ Пример: таблица содержит скидки для каждого рекламодателя в виде: ``` - +------------------+-----------------------------+------------+----------+ - | id рекламодателя | дата начала действия скидки | дата конца | величина | - +==================+=============================+============+==========+ - | 123 | 2015-01-01 | 2015-01-15 | 0.15 | - +------------------+-----------------------------+------------+----------+ - | 123 | 2015-01-16 | 2015-01-31 | 0.25 | - +------------------+-----------------------------+------------+----------+ - | 456 | 2015-01-01 | 2015-01-15 | 0.05 | - +------------------+-----------------------------+------------+----------+ ++---------------+---------------------+-------------------+--------+ +| advertiser id | discount start date | discount end date | amount | ++===============+=====================+===================+========+ +| 123 | 2015-01-01 | 2015-01-15 | 0.15 | ++---------------+---------------------+-------------------+--------+ +| 123 | 2015-01-16 | 2015-01-31 | 0.25 | ++---------------+---------------------+-------------------+--------+ +| 456 | 2015-01-01 | 2015-01-15 | 0.05 | ++---------------+---------------------+-------------------+--------+ ``` Чтобы использовать выборку по диапазонам дат, необходимо в [structure](external_dicts_dict_structure#dicts-external_dicts_dict_structure) определить элементы `range_min`, `range_max`. From 2f4d20a0cf62230783bb67a8738299f60ca63f7c Mon Sep 17 00:00:00 2001 From: BayoNet Date: Sun, 25 Mar 2018 05:37:07 +0300 Subject: [PATCH 52/70] Some typos are fixed. --- docs/ru/introduction/ya_metrika_task.md | 2 +- docs/ru/roadmap.md | 24 ++++++++++++------------ 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/ru/introduction/ya_metrika_task.md b/docs/ru/introduction/ya_metrika_task.md index 24e595b2c49..765c0450890 100644 --- a/docs/ru/introduction/ya_metrika_task.md +++ b/docs/ru/introduction/ya_metrika_task.md @@ -1,6 +1,6 @@ # Постановка задачи в Яндекс.Метрике -ClickHouse на данный момент обеспечивает рабту [Яндекс.Метрики](https://metrika.yandex.ru/), [второй крупнейшей в мире](http://w3techs.com/technologies/overview/traffic_analysis/all) платформы для веб аналитики. При более 13 триллионах записей в базе данных и более 20 миллиардах событий в сутки, ClickHouse позволяет генерировать индивидуально настроенные отчёты на лету напрямую из неагрегированных данных. +ClickHouse на данный момент обеспечивает работу [Яндекс.Метрики](https://metrika.yandex.ru/), [второй крупнейшей в мире](http://w3techs.com/technologies/overview/traffic_analysis/all) платформы для веб аналитики. При более 13 триллионах записей в базе данных и более 20 миллиардах событий в сутки, ClickHouse позволяет генерировать индивидуально настроенные отчёты на лету напрямую из неагрегированных данных. Нужно получать произвольные отчёты на основе хитов и визитов, с произвольными сегментами, задаваемыми пользователем. Данные для отчётов обновляются в реальном времени. Запросы должны выполняться сразу (в режиме онлайн). Отчёты должно быть возможно строить за произвольный период. Требуется вычислять сложные агрегаты типа количества уникальных посетителей. На данный момент (апрель 2014), каждый день в Яндекс.Метрику поступает около 12 миллиардов событий (хитов и кликов мыши). Все эти события должны быть сохранены для возможности строить произвольные отчёты. Один запрос может потребовать просканировать сотни миллионов строк за время не более нескольких секунд, или миллионы строк за время не более нескольких сотен миллисекунд. diff --git a/docs/ru/roadmap.md b/docs/ru/roadmap.md index 13c2b60c094..bbdc740bf01 100644 --- a/docs/ru/roadmap.md +++ b/docs/ru/roadmap.md @@ -5,15 +5,15 @@ ### Новая функциональность - Поддержка `UPDATE` и `DELETE`. - Многомерные и вложенные массивы. - + Это может выглядеть например так: - + ```sql CREATE TABLE t ( - x Array(Array(String)), + x Array(Array(String)), z Nested( - x Array(String), + x Array(String), y Nested(...)) ) ENGINE = MergeTree ORDER BY x @@ -24,7 +24,7 @@ ENGINE = MergeTree ORDER BY x Внешние таблицы можно интрегрировать в ClickHouse с помощью внешних словарей. Новая функциональность станет более удобной альтернативой для подключения внешних таблиц. ```sql -SELECT ... +SELECT ... FROM mysql('host:port', 'db', 'table', 'user', 'password')` ``` @@ -40,7 +40,7 @@ INSERT INTO t SELECT * FROM remote(...) `. - O_DIRECT for merges. Улучшит производительность кэша операционной системы, а также производительность 'горячих' запросов. - + ## Q2 2018 ### Новая функциональность @@ -56,23 +56,23 @@ INSERT INTO t SELECT * FROM remote(...) `. - Настраиваемые кодировки и сжатие для каждого столбца в отдельности. Сейчас, ClickHouse поддерживает сжатие столбцов с помощью LZ4 и ZSTD, и настройки сжатия глобальные (смотрите статью [Compression in ClickHouse](https://www.altinity.com/blog/2017/11/21/compression-in-clickhouse)). Поколоночное сжатие и кодирование обеспечит более эффективное хранение данных, что в свою очередь ускорит выполнение запросов. - + - Хранение данных на нескольких дисках на одном сервере. Реализация это функциональности упростит расширение дискового пространства, поскольку можно будет использовать различные дисковые системы для разных баз данных или таблиц. Сейчас, пользователи вынуждены использовать символические ссылки, если базы данных и таблицы должны храниться на другом диске. - + ### Улучшения Планируется множество улучшений и исправлений в системе выполнения запросов. Например: - Использование индекса для `in (subquery)`. - Сейчас, индекс не используется, что приводит с снижению производительности. - + Сейчас, индекс не используется, что приводит к снижению производительности. + - Передача предикатов из `where` в подзапросы, а также передача предикатов в представления. Передача предикатов необходима, поскольку представление изменяется поздапросом. Сейчас производительность фильтров для представлений низкая, представления не могут использовать первичный ключ оригинальной таблицы, что делает представления для больших таблиц бесполезными. - + - Оптимизация операций с ветвлением (тернарный оператор, if, multiIf). Сейчас, ClickHouse выполняет все ветви, даже если в этом нет необходимости. @@ -88,7 +88,7 @@ INSERT INTO t SELECT * FROM remote(...) `. - Пулы ресурсов для выполнения запросов. Позволят более эффективно управлять нагрузкой. - + - Синтаксис ANSI SQL JOIN. Улучшит совместимость ClickHouse со множеством SQL-инструментов. From 69004325275cfdf6d8d35a7350c6bdc4dac76f36 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Sun, 25 Mar 2018 05:16:13 +0300 Subject: [PATCH 53/70] mysql.md is not translated yet --- docs/mkdocs_en.yml | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/mkdocs_en.yml b/docs/mkdocs_en.yml index 012d498f3e2..eeedc71a79b 100644 --- a/docs/mkdocs_en.yml +++ b/docs/mkdocs_en.yml @@ -96,7 +96,6 @@ pages: - 'View': 'table_engines/view.md' - 'MaterializedView': 'table_engines/materializedview.md' - 'Kafka': 'table_engines/kafka.md' - - 'MySQL': 'table_engines/mysql.md' - 'External data for query processing': 'table_engines/external_data.md' - 'System tables': From 474fc1dbb623fe8da43dbbd7b301d7e610da3a43 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Sun, 25 Mar 2018 06:08:08 +0300 Subject: [PATCH 54/70] Some more typos are fixed. --- docs/en/dicts/external_dicts_dict_layout.md | 63 +++++++++++++++++++ docs/en/index.md | 3 +- .../en/operations/server_settings/settings.md | 2 +- docs/en/query_language/queries.md | 3 +- 4 files changed, 66 insertions(+), 5 deletions(-) diff --git a/docs/en/dicts/external_dicts_dict_layout.md b/docs/en/dicts/external_dicts_dict_layout.md index 8b7cad24b65..aae90b5724f 100755 --- a/docs/en/dicts/external_dicts_dict_layout.md +++ b/docs/en/dicts/external_dicts_dict_layout.md @@ -46,6 +46,7 @@ The configuration looks like this: - [range_hashed](#dicts-external_dicts_dict_layout-range_hashed) - [complex_key_hashed](#dicts-external_dicts_dict_layout-complex_key_hashed) - [complex_key_cache](#dicts-external_dicts_dict_layout-complex_key_cache) +- [ip_trie](#dicts-external_dicts_dict_layout-ip_trie) @@ -227,3 +228,65 @@ Do not use ClickHouse as a source, because it is slow to process queries with ra ### complex_key_cache This type of storage is for use with composite [keys](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure). Similar to `cache`. + + + +### ip_trie + + +The table stores IP prefixes for each key (IP address), which makes it possible to map IP addresses to metadata such as ASN or threat score. + +Example: in the table there are prefixes matches to AS number and country: + +``` + +-----------------+-------+--------+ + | prefix | asn | cca2 | + +=================+=======+========+ + | 202.79.32.0/20 | 17501 | NP | + +-----------------+-------+--------+ + | 2620:0:870::/48 | 3856 | US | + +-----------------+-------+--------+ + | 2a02:6b8:1::/48 | 13238 | RU | + +-----------------+-------+--------+ + | 2001:db8::/32 | 65536 | ZZ | + +-----------------+-------+--------+ +``` + +When using such a layout, the structure should have the "key" element. + +Example: + +```xml + + + + prefix + String + + + + asn + UInt32 + + + + cca2 + String + ?? + + ... +``` + +These key must have only one attribute of type String, containing a valid IP prefix. Other types are not yet supported. + +For querying, same functions (dictGetT with tuple) as for complex key dictionaries have to be used: + + dictGetT('dict_name', 'attr_name', tuple(ip)) + +The function accepts either UInt32 for IPv4 address or FixedString(16) for IPv6 address in wire format: + + dictGetString('prefix', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) + +No other type is supported. The function returns attribute for a prefix matching the given IP address. If there are overlapping prefixes, the most specific one is returned. + +The data is stored currently in a bitwise trie, it has to fit in memory. diff --git a/docs/en/index.md b/docs/en/index.md index 72efa70802b..cc9c806fe50 100755 --- a/docs/en/index.md +++ b/docs/en/index.md @@ -39,7 +39,7 @@ We'll say that the following is true for the OLAP (online analytical processing) - Data is updated in fairly large batches (> 1000 rows), not by single rows; or it is not updated at all. - Data is added to the DB but is not modified. - For reads, quite a large number of rows are extracted from the DB, but only a small subset of columns. -- Tables are "wide," meaning they contain a large number of columns. +- Tables are "wide", meaning they contain a large number of columns. - Queries are relatively rare (usually hundreds of queries per server or less per second). - For simple queries, latencies around 50 ms are allowed. - Column values are fairly small: numbers and short strings (for example, 60 bytes per URL). @@ -120,4 +120,3 @@ There are two ways to do this: This is not done in "normal" databases, because it doesn't make sense when running simple queries. However, there are exceptions. For example, MemSQL uses code generation to reduce latency when processing SQL queries. (For comparison, analytical DBMSs require optimization of throughput, not latency.) Note that for CPU efficiency, the query language must be declarative (SQL or MDX), or at least a vector (J, K). The query should only contain implicit loops, allowing for optimization. - diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index e1575df2f88..d65b15d377d 100755 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -100,7 +100,7 @@ Path: - Specify the absolute path or the path relative to the server config file. - The path can contain wildcards \* and ?. -See also "[External dictionaries]("./../dicts/external_dicts.md#dicts-external_dicts)". +See also "[External dictionaries](../../dicts/external_dicts.md#dicts-external_dicts)". **Example** diff --git a/docs/en/query_language/queries.md b/docs/en/query_language/queries.md index d235945a646..cf33c7994c5 100755 --- a/docs/en/query_language/queries.md +++ b/docs/en/query_language/queries.md @@ -323,7 +323,7 @@ Here, `20140317_20140323_2_2_0` and ` 20140317_20140323_4_4_0` are the directori Let's break down the name of the first part: `20140317_20140323_2_2_0`. - `20140317` is the minimum date of the data in the chunk. -- `20140323` is the maximum data of the data in the chunk. +- `20140323` is the maximum date of the data in the chunk. - `2` is the minimum number of the data block. - `2` is the maximum number of the data block. - `0` is the chunk level (the depth of the merge tree it is formed from). @@ -1506,4 +1506,3 @@ The response contains the `kill_status` column, which can take the following val 3. The other values ​​explain why the query can't be terminated. A test query (`TEST`) only checks the user's rights and displays a list of queries to terminate. - From 6881d84bc2f3e67b7fef0614fc0d4021eccd4293 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Mon, 26 Mar 2018 16:16:59 +0300 Subject: [PATCH 55/70] External editions are revised. English translation is actualised from 02.03.2018 version up to 26.03.2018. --- docs/en/dicts/external_dicts_dict.md | 5 ++- docs/en/dicts/external_dicts_dict_layout.md | 36 +++++++++---------- .../en/dicts/external_dicts_dict_structure.md | 2 +- docs/en/functions/array_functions.md | 2 +- docs/en/functions/json_functions.md | 2 +- docs/en/getting_started/index.md | 10 ++---- docs/en/interfaces/http_interface.md | 16 ++++++--- .../en/operations/server_settings/settings.md | 4 +-- docs/en/query_language/queries.md | 19 +++++----- docs/ru/agg_functions/reference.md | 8 ++--- .../operations/settings/query_complexity.md | 2 +- docs/ru/query_language/queries.md | 10 +++--- 12 files changed, 59 insertions(+), 57 deletions(-) diff --git a/docs/en/dicts/external_dicts_dict.md b/docs/en/dicts/external_dicts_dict.md index 6d2f4128704..0e9b6f578b4 100755 --- a/docs/en/dicts/external_dicts_dict.md +++ b/docs/en/dicts/external_dicts_dict.md @@ -27,8 +27,7 @@ The dictionary configuration has the following structure: ``` - name – The identifier that can be used to access the dictionary. Use the characters `[a-zA-Z0-9_\-]`. -- [source](external_dicts_dict_sources.html/#dicts-external_dicts_dict_sources) — Source of the dictionary . +- [source](external_dicts_dict_sources.md/#dicts-external_dicts_dict_sources) — Source of the dictionary . - [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) — Dictionary layout in memory. -- [source](external_dicts_dict_sources.html/#dicts-external_dicts_dict_sources) — Structure of the dictionary . A key and attributes that can be retrieved by this key. +- [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure) — Structure of the dictionary . A key and attributes that can be retrieved by this key. - [lifetime](external_dicts_dict_lifetime.md#dicts-external_dicts_dict_lifetime) — Frequency of dictionary updates. - diff --git a/docs/en/dicts/external_dicts_dict_layout.md b/docs/en/dicts/external_dicts_dict_layout.md index aae90b5724f..ad635db94f5 100755 --- a/docs/en/dicts/external_dicts_dict_layout.md +++ b/docs/en/dicts/external_dicts_dict_layout.md @@ -2,11 +2,11 @@ # Storing dictionaries in memory -There are a [variety of ways](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-manner) to store dictionaries in memory. +There are [many different ways](external_dicts_dict_layout#dicts-external_dicts_dict_layout-manner) to store dictionaries in memory. -We recommend [flat](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-flat), [hashed](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-hashed)and[complex_key_hashed](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-complex_key_hashed). which provide optimal processing speed. +We recommend [flat](external_dicts_dict_layout#dicts-external_dicts_dict_layout-flat), [hashed](external_dicts_dict_layout#dicts-external_dicts_dict_layout-hashed), and [complex_key_hashed](external_dicts_dict_layout#dicts-external_dicts_dict_layout-complex_key_hashed). which provide optimal processing speed. -Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more in the section " [cache](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout-cache)". +Caching is not recommended because of potentially poor performance and difficulties in selecting optimal parameters. Read more about this in the "[cache](external_dicts_dict_layout#dicts-external_dicts_dict_layout-cache)" section. There are several ways to improve dictionary performance: @@ -88,7 +88,7 @@ Configuration example: ### complex_key_hashed -This type is for use with composite [keys](external_dicts_dict_structure.md/#dicts-external_dicts_dict_structure). Similar to `hashed`. +This type of storage is designed for use with compound [keys](external_dicts_dict_structure#dicts-external_dicts_dict_structure). It is similar to hashed. Configuration example: @@ -109,18 +109,18 @@ This storage method works the same way as hashed and allows using date/time rang Example: The table contains discounts for each advertiser in the format: ``` -+---------------+---------------------+-------------------+--------+ -| advertiser id | discount start date | discount end date | amount | -+===============+=====================+===================+========+ -| 123 | 2015-01-01 | 2015-01-15 | 0.15 | -+---------------+---------------------+-------------------+--------+ -| 123 | 2015-01-16 | 2015-01-31 | 0.25 | -+---------------+---------------------+-------------------+--------+ -| 456 | 2015-01-01 | 2015-01-15 | 0.05 | -+---------------+---------------------+-------------------+--------+ + +---------------+---------------------+-------------------+--------+ + | advertiser id | discount start date | discount end date | amount | + +===============+=====================+===================+========+ + | 123 | 2015-01-01 | 2015-01-15 | 0.15 | + +---------------+---------------------+-------------------+--------+ + | 123 | 2015-01-16 | 2015-01-31 | 0.25 | + +---------------+---------------------+-------------------+--------+ + | 456 | 2015-01-01 | 2015-01-15 | 0.05 | + +---------------+---------------------+-------------------+--------+ ``` -To use a sample for date ranges, define the `range_min` and `range_max` elements in the [structure](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure). +To use a sample for date ranges, define `range_min` and `range_max` in [structure](external_dicts_dict_structure#dicts-external_dicts_dict_structure). Example: @@ -197,15 +197,15 @@ This is the least effective of all the ways to store dictionaries. The speed of To improve cache performance, use a subquery with ` LIMIT`, and call the function with the dictionary externally. -Supported [sources](external_dicts_dict_sources.md#dicts-external_dicts_dict_sources): MySQL, ClickHouse, executable, HTTP. +Supported [sources](external_dicts_dict_sources#dicts-external_dicts_dict_sources): MySQL, ClickHouse, executable, HTTP. Example of settings: ```xml - - 1000000000 + + 1000000000 ``` @@ -227,7 +227,7 @@ Do not use ClickHouse as a source, because it is slow to process queries with ra ### complex_key_cache -This type of storage is for use with composite [keys](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure). Similar to `cache`. +This type of storage is designed for use with compound [keys](external_dicts_dict_structure#dicts-external_dicts_dict_structure). Similar to `cache`. diff --git a/docs/en/dicts/external_dicts_dict_structure.md b/docs/en/dicts/external_dicts_dict_structure.md index 2542af00ec6..b6038010623 100755 --- a/docs/en/dicts/external_dicts_dict_structure.md +++ b/docs/en/dicts/external_dicts_dict_structure.md @@ -66,7 +66,7 @@ Configuration fields: The key can be a `tuple` from any types of fields. The [layout](external_dicts_dict_layout.md#dicts-external_dicts_dict_layout) in this case must be `complex_key_hashed` or `complex_key_cache`.
-A composite key can consist of a single element. This makes it possible to use a string as the key, for instance. +A composite key can consist of a single element. This makes it possible to use a string as the key, for instance.
The key structure is set in the element ``. Key fields are specified in the same format as the dictionary [attributes](external_dicts_dict_structure.md#dicts-external_dicts_dict_structure-attributes). Example: diff --git a/docs/en/functions/array_functions.md b/docs/en/functions/array_functions.md index 232f6a20427..6993132f423 100755 --- a/docs/en/functions/array_functions.md +++ b/docs/en/functions/array_functions.md @@ -39,7 +39,7 @@ Accepts an empty array and returns a one-element array that is equal to the defa Returns an array of numbers from 0 to N-1. Just in case, an exception is thrown if arrays with a total length of more than 100,000,000 elements are created in a data block. -## array(x1, ...), оператор \[x1, ...\] +## array(x1, ...), operator \[x1, ...\] Creates an array from the function arguments. The arguments must be constants and have types that have the smallest common type. At least one argument must be passed, because otherwise it isn't clear which type of array to create. That is, you can't use this function to create an empty array (to do that, use the 'emptyArray\*' function described above). diff --git a/docs/en/functions/json_functions.md b/docs/en/functions/json_functions.md index 90a2ddc47dd..70f66d86b61 100755 --- a/docs/en/functions/json_functions.md +++ b/docs/en/functions/json_functions.md @@ -5,7 +5,7 @@ In Yandex.Metrica, JSON is transmitted by users as session parameters. There are The following assumptions are made: 1. The field name (function argument) must be a constant. -2. The field name is somehow canonically encoded in JSON. For example: `visitParamHas('{"abc":"def"}', 'abc') = 1`, но `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` +2. The field name is somehow canonically encoded in JSON. For example: `visitParamHas('{"abc":"def"}', 'abc') = 1`, but `visitParamHas('{"\\u0061\\u0062\\u0063":"def"}', 'abc') = 0` 3. Fields are searched for on any nesting level, indiscriminately. If there are multiple matching fields, the first occurrence is used. 4. The JSON doesn't have space characters outside of string literals. diff --git a/docs/en/getting_started/index.md b/docs/en/getting_started/index.md index 07d0d91a224..d3e9ea03915 100755 --- a/docs/en/getting_started/index.md +++ b/docs/en/getting_started/index.md @@ -16,15 +16,14 @@ The terminal must use UTF-8 encoding (the default in Ubuntu). For testing and development, the system can be installed on a single server or on a desktop computer. -### Installing from packages +### Installing from packages Debian/Ubuntu In `/etc/apt/sources.list` (or in a separate `/etc/apt/sources.list.d/clickhouse.list` file), add the repository: ```text -deb http://repo.yandex.ru/clickhouse/trusty stable main +deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ ``` -On other versions of Ubuntu, replace `trusty` with `xenial` or `precise`. If you want to use the most recent test version, replace 'stable' with 'testing'. Then run: @@ -36,9 +35,7 @@ sudo apt-get install clickhouse-client clickhouse-server-common ``` You can also download and install packages manually from here: - - - + ClickHouse contains access restriction settings. They are located in the 'users.xml' file (next to 'config.xml'). By default, access is allowed from anywhere for the 'default' user, without a password. See 'user/default/networks'. @@ -137,4 +134,3 @@ SELECT 1 **Congratulations, the system works!** To continue experimenting, you can try to download from the test data sets. - diff --git a/docs/en/interfaces/http_interface.md b/docs/en/interfaces/http_interface.md index 38a70feef46..8c223cf69cf 100755 --- a/docs/en/interfaces/http_interface.md +++ b/docs/en/interfaces/http_interface.md @@ -37,8 +37,7 @@ Date: Fri, 16 Nov 2012 19:21:50 GMT 1 ``` -As you can see, curl is somewhat inconvenient in that spaces must be URL escaped. -Although wget escapes everything itself, we don't recommend using it because it doesn't work well over HTTP 1.1 when using keep-alive and Transfer-Encoding: chunked. +As you can see, curl is somewhat inconvenient in that spaces must be URL escaped.Although wget escapes everything itself, we don't recommend using it because it doesn't work well over HTTP 1.1 when using keep-alive and Transfer-Encoding: chunked. ```bash $ echo 'SELECT 1' | curl 'http://localhost:8123/' --data-binary @- @@ -131,11 +130,15 @@ POST 'http://localhost:8123/?query=DROP TABLE t' For successful requests that don't return a data table, an empty response body is returned. -You can use compression when transmitting data. The compressed data has a non-standard format, and you will need to use the special compressor program to work with it (sudo apt-get install compressor-metrika-yandex). +You can use compression when transmitting data. +For using ClickHouse internal compression format, and you will need to use the special compressor program to work with it (sudo apt-get install compressor-metrika-yandex). If you specified 'compress=1' in the URL, the server will compress the data it sends you. If you specified 'decompress=1' in the URL, the server will decompress the same data that you pass in the POST method. +Also standard gzip-based HTTP compression can be used. To send gzip compressed POST data just add `Content-Encoding: gzip` to request headers, and gzip POST body. +To get response compressed, you need to add `Accept-Encoding: gzip` to request headers, and turn on ClickHouse setting called `enable_http_compression`. + You can use this to reduce network traffic when transmitting a large amount of data, or for creating dumps that are immediately compressed. You can use the 'database' URL parameter to specify the default database. @@ -191,7 +194,11 @@ $ echo 'SELECT number FROM system.numbers LIMIT 10' | curl 'http://localhost:812 For information about other parameters, see the section "SET". -In contrast to the native interface, the HTTP interface does not support the concept of sessions or session settings, does not allow aborting a query (to be exact, it allows this in only a few cases), and does not show the progress of query processing. Parsing and data formatting are performed on the server side, and using the network might be ineffective. +You can use ClickHouse sessions in the HTTP protocol. To do this, you need to specify the `session_id` GET parameter in HTTP request. You can use any alphanumeric string as a session_id. By default session will be timed out after 60 seconds of inactivity. You can change that by setting `default_session_timeout` in server config file, or by adding GET parameter `session_timeout`. You can also check the status of the session by using GET parameter `session_check=1`. When using sessions you can't run 2 queries with the same session_id simultaneously. + +You can get the progress of query execution in X-ClickHouse-Progress headers, by enabling setting send_progress_in_http_headers. + +Running query are not aborted automatically after closing HTTP connection. Parsing and data formatting are performed on the server side, and using the network might be ineffective. The optional 'query_id' parameter can be passed as the query ID (any string). For more information, see the section "Settings, replace_running_query". The optional 'quota_key' parameter can be passed as the quota key (any string). For more information, see the section "Quotas". @@ -213,4 +220,3 @@ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&wa ``` Use buffering to avoid situations where a query processing error occurred after the response code and HTTP headers were sent to the client. In this situation, an error message is written at the end of the response body, and on the client side, the error can only be detected at the parsing stage. - diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index d65b15d377d..e9916b9a836 100755 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -440,14 +440,14 @@ For more information, see the MergeTreeSettings.h header file. SSL client/server configuration. -Support for SSL is provided by the `` libpoco`` library. The interface is described in the file [SSLManager.h](https://github.com/yandex/ClickHouse/blob/master/contrib/libpoco/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h) +Support for SSL is provided by the `` libpoco`` library. The interface is described in the file [SSLManager.h](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/SSLManager.h) Keys for server/client settings: - privateKeyFile – The path to the file with the secret key of the PEM certificate. The file may contain a key and certificate at the same time. - certificateFile – The path to the client/server certificate file in PEM format. You can omit it if `` privateKeyFile`` contains the certificate. - caConfig – The path to the file or directory that contains trusted root certificates. -- verificationMode – The method for checking the node's certificates. Details are in the description of the [Context](https://github.com/yandex/ClickHouse/blob/master/contrib/libpoco/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: ``none``, ``relaxed``, ``strict``, ``once``. +- verificationMode – The method for checking the node's certificates. Details are in the description of the [Context](https://github.com/ClickHouse-Extras/poco/blob/master/NetSSL_OpenSSL/include/Poco/Net/Context.h) class. Possible values: ``none``, ``relaxed``, ``strict``, ``once``. - verificationDepth – The maximum length of the verification chain. Verification will fail if the certificate chain length exceeds the set value. - loadDefaultCAFile – Indicates that built-in CA certificates for OpenSSL will be used. Acceptable values: `` true``, `` false``. | - cipherList - Поддерживаемые OpenSSL-шифры. For example: `` ALL:!ADH:!LOW:!EXP:!MD5:@STRENGTH``. diff --git a/docs/en/query_language/queries.md b/docs/en/query_language/queries.md index cf33c7994c5..a8503a91bc2 100755 --- a/docs/en/query_language/queries.md +++ b/docs/en/query_language/queries.md @@ -1434,7 +1434,7 @@ and the result will be put in a temporary table in RAM. Then the request will be SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL IN _data1 ``` -and the temporary table '_data1' will be sent to every remote server together with the query (the name of the temporary table is implementation-defined). +and the temporary table `_data1` will be sent to every remote server together with the query (the name of the temporary table is implementation-defined). This is more optimal than using the normal IN. However, keep the following points in mind: @@ -1476,28 +1476,29 @@ In all other cases, we don't recommend using the asterisk, since it only gives y ## KILL QUERY ```sql -KILL QUERY WHERE [SYNC|ASYNC|TEST] [FORMAT format] +KILL QUERY + WHERE + [SYNC|ASYNC|TEST] + [FORMAT format] ``` Attempts to terminate queries currently running. -The queries to terminate are selected from the system.processes table for which expression_for_system.processes is true. +The queries to terminate are selected from the system.processes table for which `WHERE` expression is true. Examples: ```sql +-- Terminates all queries with the specified query_id. KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90' -``` -Terminates all queries with the specified query_id. - -```sql +-- Synchronously terminates all queries run by `username`. KILL QUERY WHERE user='username' SYNC ``` -Synchronously terminates all queries run by `username`. - Readonly-users can only terminate their own requests. + By default, the asynchronous version of queries is used (`ASYNC`), which terminates without waiting for queries to complete. + The synchronous version (`SYNC`) waits for all queries to be completed and displays information about each process as it terminates. The response contains the `kill_status` column, which can take the following values: diff --git a/docs/ru/agg_functions/reference.md b/docs/ru/agg_functions/reference.md index b31d4b5496b..6b30d771dd9 100644 --- a/docs/ru/agg_functions/reference.md +++ b/docs/ru/agg_functions/reference.md @@ -22,7 +22,7 @@ При наличии в запросе `SELECT` секции `GROUP BY` или хотя бы одной агрегатной функции, ClickHouse (в отличие от, например, MySQL) требует, чтобы все выражения в секциях `SELECT`, `HAVING`, `ORDER BY` вычислялись из ключей или из агрегатных функций. То есть, каждый выбираемый из таблицы столбец, должен использоваться либо в ключах, либо внутри агрегатных функций. Чтобы получить поведение, как в MySQL, вы можете поместить остальные столбцы в агрегатную функцию `any`. -## anyHeavy +## anyHeavy(x) Выбирает часто встречающееся значение с помощью алгоритма "[heavy hitters](http://www.cs.umd.edu/~samir/498/karp.pdf)". Если существует значение, которое встречается чаще, чем в половине случаев, в каждом потоке выполнения запроса, то возвращается данное значение. В общем случае, результат недетерминирован. @@ -185,7 +185,7 @@ GROUP BY timeslot -## groupArrayInsertAt +## groupArrayInsertAt(x) Вставляет в массив значение в заданную позицию. @@ -281,7 +281,7 @@ GROUP BY timeslot Результат зависит от порядка выполнения запроса, и является недетерминированным. -## median +## median(x) Для всех quantile-функций, также присутствуют соответствующие median-функции: `median`, `medianDeterministic`, `medianTiming`, `medianTimingWeighted`, `medianExact`, `medianExactWeighted`, `medianTDigest`. Они являются синонимами и их поведение ничем не отличается. @@ -315,7 +315,7 @@ GROUP BY timeslot Результат равен квадратному корню от `varPop(x)`. -## topK +## topK(N)(column) Возвращает массив наиболее часто встречающихся значений в указанном столбце. Результирующий массив упорядочен по убыванию частоты значения (не по самим значениям). diff --git a/docs/ru/operations/settings/query_complexity.md b/docs/ru/operations/settings/query_complexity.md index afbba3bc688..9b36cff27ad 100644 --- a/docs/ru/operations/settings/query_complexity.md +++ b/docs/ru/operations/settings/query_complexity.md @@ -33,7 +33,7 @@ Максимальный возможный объем оперативной памяти для выполнения запроса на одном сервере. -В конфигурационном файле по-умолчанию, ограничение равно 10 ГБ. +В конфигурационном файле по умолчанию, ограничение равно 10 ГБ. Настройка не учитывает объём свободной памяти или общий объём памяти на машине. Ограничение действует на один запрос, в пределах одного сервера. diff --git a/docs/ru/query_language/queries.md b/docs/ru/query_language/queries.md index c5a1d46273e..61957616f2c 100644 --- a/docs/ru/query_language/queries.md +++ b/docs/ru/query_language/queries.md @@ -180,7 +180,7 @@ DROP DATABASE [IF EXISTS] db [ON CLUSTER cluster] Если указано `IF EXISTS` - не выдавать ошибку, если база данных не существует. ```sql -DROP TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] +DROP [TEMPORARY] TABLE [IF EXISTS] [db.]name [ON CLUSTER cluster] ``` Удаляет таблицу. @@ -444,7 +444,7 @@ SHOW DATABASES [INTO OUTFILE filename] [FORMAT format] ## SHOW TABLES ```sql -SHOW TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format] +SHOW [TEMPORARY] TABLES [FROM db] [LIKE 'pattern'] [INTO OUTFILE filename] [FORMAT format] ``` Выводит список таблиц @@ -491,7 +491,7 @@ watch -n1 "clickhouse-client --query='SHOW PROCESSLIST'" ## SHOW CREATE TABLE ```sql -SHOW CREATE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] +SHOW CREATE [TEMPORARY] TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ``` Возвращает один столбец statement типа `String`, содержащий одно значение - запрос `CREATE`, с помощью которого создана указанная таблица. @@ -509,7 +509,7 @@ DESC|DESCRIBE TABLE [db.]table [INTO OUTFILE filename] [FORMAT format] ## EXISTS ```sql -EXISTS TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] +EXISTS [TEMPORARY] TABLE [db.]name [INTO OUTFILE filename] [FORMAT format] ``` Возвращает один столбец типа `UInt8`, содержащий одно значение - `0`, если таблицы или БД не существует и `1`, если таблица в указанной БД существует. @@ -1430,7 +1430,7 @@ SELECT UserID FROM distributed_table WHERE CounterID = 34 SELECT uniq(UserID) FROM local_table WHERE CounterID = 101500 AND UserID GLOBAL IN _data1 ``` -, и вместе с запросом, на каждый удалённый сервер будет отправлена временная таблица _data1 (имя временной таблицы - implementation defined). +, и вместе с запросом, на каждый удалённый сервер будет отправлена временная таблица `_data1` (имя временной таблицы - implementation defined). Это гораздо более оптимально, чем при использовании обычного IN. Но при этом, следует помнить о нескольких вещах: From 424c62e4131ecd03584485ec773a05e28a184626 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E7=A3=8A?= <2217232293@qq.com> Date: Fri, 30 Mar 2018 02:51:43 -0500 Subject: [PATCH 56/70] update README.md --- README.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 8f93b8d9af1..e5d998b54e7 100644 --- a/README.md +++ b/README.md @@ -1,5 +1,6 @@ +# ClickHouse ClickHouse is an open-source column-oriented database management system that allows generating analytical data reports in real time. -[Read more...](https://clickhouse.yandex/) +Learn more about ClickHouse at [https://clickhouse.yandex/](https://clickhouse.yandex/) [![Build Status](https://travis-ci.org/yandex/ClickHouse.svg?branch=master)](https://travis-ci.org/yandex/ClickHouse) From ce23653caddae56ac29d013397148c34e33f5a85 Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 30 Mar 2018 05:42:06 -0700 Subject: [PATCH 57/70] Fix build with old poco (#2144) --- dbms/src/Server/Server.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 0cba1117e60..5c692462058 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -369,10 +369,14 @@ int Server::main(const std::vector & /*args*/) if (secure) /// Bug in old poco, listen() after bind() with reusePort param will fail because have no implementation in SecureServerSocketImpl /// https://github.com/pocoproject/poco/pull/2257 - socket.bind(address, true); + socket.bind(address, /* reuseAddress = */ true); else #endif +#if POCO_VERSION < 0x01080000 + socket.bind(address, /* reuseAddress = */ true); +#else socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ config().getBool("listen_reuse_port", false)); +#endif socket.listen(/* backlog = */ config().getUInt("listen_backlog", 64)); From 5fdd50c2a22957b56eb3348ae7c96450b2f5178e Mon Sep 17 00:00:00 2001 From: Alexey Zatelepin Date: Fri, 30 Mar 2018 16:33:43 +0300 Subject: [PATCH 58/70] process included config node once more if include happened [#MTRSADMIN-3894] --- dbms/src/Common/Config/ConfigProcessor.cpp | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index fdf392bb03f..6100173c318 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -276,6 +276,8 @@ void ConfigProcessor::doIncludesRecursive( /// Replace the original contents, not add to it. bool replace = attributes->getNamedItem("replace"); + bool included_something = false; + auto process_include = [&](const Node * include_attr, const std::function & get_node, const char * error_msg) { std::string name = include_attr->getNodeValue(); @@ -316,6 +318,8 @@ void ConfigProcessor::doIncludesRecursive( { element->setAttributeNode(dynamic_cast(config->importNode(from_attrs->item(i), true))); } + + included_something = true; } }; @@ -348,11 +352,14 @@ void ConfigProcessor::doIncludesRecursive( } } - NodeListPtr children = node->childNodes(); - Node * child = nullptr; - for (size_t i = 0; (child = children->item(i)); ++i) + if (included_something) + doIncludesRecursive(config, include_from, node, zk_node_cache, contributing_zk_paths); + else { - doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths); + NodeListPtr children = node->childNodes(); + Node * child = nullptr; + for (size_t i = 0; (child = children->item(i)); ++i) + doIncludesRecursive(config, include_from, child, zk_node_cache, contributing_zk_paths); } } From b3e8949d2a589a9dc42280bf2bc43a863d0b7f9a Mon Sep 17 00:00:00 2001 From: proller Date: Fri, 30 Mar 2018 16:58:00 +0300 Subject: [PATCH 59/70] CLICKHOUSE-3675 Add glibc version test Fix zookeeper tools compile --- contrib/zookeeper-cmake/CMakeLists.txt | 2 ++ dbms/src/Server/CMakeLists.txt | 3 +++ 2 files changed, 5 insertions(+) diff --git a/contrib/zookeeper-cmake/CMakeLists.txt b/contrib/zookeeper-cmake/CMakeLists.txt index 1211f1f8e14..df8859ffbeb 100644 --- a/contrib/zookeeper-cmake/CMakeLists.txt +++ b/contrib/zookeeper-cmake/CMakeLists.txt @@ -192,9 +192,11 @@ endif() # cli executable add_executable(cli ${LIBRARY_DIR}/src/cli.c) target_link_libraries(cli zookeeper) +target_link_libraries(cli $<$:m>) # load_gen executable if(WANT_SYNCAPI AND NOT WIN32) add_executable(load_gen ${LIBRARY_DIR}/src/load_gen.c) target_link_libraries(load_gen zookeeper) + target_link_libraries(load_gen $<$:m>) endif() diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index d268851f63a..a153d3c932a 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -192,6 +192,9 @@ else () add_custom_target (clickhouse-bundle ALL DEPENDS ${CLICKHOUSE_BUNDLE}) endif () +set (GLIBC_MAX_REQUIRED 2.4) +add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort -r | perl -lnE 'exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'") + install ( FILES config.xml users.xml DESTINATION ${CLICKHOUSE_ETC_DIR}/clickhouse-server From c898986650d9d2231b98726450894a7cf58bd34b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E8=B0=A2=E7=A3=8A?= <2217232293@qq.com> Date: Sat, 31 Mar 2018 12:54:06 -0500 Subject: [PATCH 60/70] change the return type of funciton countEqual the maximum length of the array is defined as UInt64. the maximum of `UInt32` just 4294967295. it's better to define the return type of the function `countEqual` as `UInt64`. --- dbms/src/Functions/FunctionsArray.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsArray.h b/dbms/src/Functions/FunctionsArray.h index 322a5a9affa..dca9bdaa902 100644 --- a/dbms/src/Functions/FunctionsArray.h +++ b/dbms/src/Functions/FunctionsArray.h @@ -179,7 +179,7 @@ struct IndexIdentity /// For countEqual. struct IndexCount { - using ResultType = UInt32; + using ResultType = UInt64; static bool apply(size_t, ResultType & current) { ++current; return true; } }; From 802cac725f3cc052034f7e69141c496559e57760 Mon Sep 17 00:00:00 2001 From: proller Date: Sun, 1 Apr 2018 05:33:39 -0700 Subject: [PATCH 61/70] Debian: Rename packages (#678) * Debian: rename packages * wip * wip * wip * move limits.d/clickhouse.conf to common * wip * fix * bad clang * update changelog * more Provides: Conflicts: * clickhouse-zookeeper-cli [#CLICKHOUSE-2947] * fix utils desc * Fix docs * use cmake links * fix doc * Fix test install * Test me * Test me * Test me. * Dont use pre-depends * fix * Minimize control changes * del * debian/clickhouse-common.install -> debian/clickhouse-common-static.install * debian/clickhouse-common.install -> debian/clickhouse-common-static.install --- copy_headers.sh | 2 +- debian/clickhouse-client.install | 12 +++---- debian/clickhouse-common-static.install | 3 ++ debian/clickhouse-server-base.install | 11 ------ debian/clickhouse-server-common.install | 2 -- debian/clickhouse-server-metrika.links | 1 + ...erver-base.docs => clickhouse-server.docs} | 0 debian/clickhouse-server.install | 8 +++++ ...se.postinst => clickhouse-server.postinst} | 0 ...base.preinst => clickhouse-server.preinst} | 0 ...ver-base.prerm => clickhouse-server.prerm} | 0 debian/control | 35 ++++++++++++------- debian/rules | 14 +++----- debian/source/options | 4 +-- docker/server/Dockerfile | 2 +- docs/en/getting_started/index.md | 2 +- docs/ru/getting_started/index.md | 2 +- website/deprecated/reference_en.html | 4 +-- website/deprecated/reference_ru.html | 4 +-- website/index.html | 2 +- website/tutorial.html | 2 +- 21 files changed, 56 insertions(+), 54 deletions(-) create mode 100644 debian/clickhouse-common-static.install delete mode 100644 debian/clickhouse-server-base.install delete mode 100644 debian/clickhouse-server-common.install create mode 100644 debian/clickhouse-server-metrika.links rename debian/{clickhouse-server-base.docs => clickhouse-server.docs} (100%) create mode 100644 debian/clickhouse-server.install rename debian/{clickhouse-server-base.postinst => clickhouse-server.postinst} (100%) rename debian/{clickhouse-server-base.preinst => clickhouse-server.preinst} (100%) rename debian/{clickhouse-server-base.prerm => clickhouse-server.prerm} (100%) diff --git a/copy_headers.sh b/copy_headers.sh index 7c106f2c3be..3396a1dec69 100755 --- a/copy_headers.sh +++ b/copy_headers.sh @@ -8,7 +8,7 @@ # Требуется clang, желательно наиболее свежий (trunk). # # Используется при сборке пакетов. -# Заголовочные файлы записываются в пакет clickhouse-server-base, в директорию /usr/share/clickhouse/headers. +# Заголовочные файлы записываются в пакет clickhouse-common, в директорию /usr/share/clickhouse/headers. # # Если вы хотите установить их самостоятельно, без сборки пакета, # чтобы clickhouse-server видел их там, где ожидается, выполните: diff --git a/debian/clickhouse-client.install b/debian/clickhouse-client.install index e41d7a4382f..561c0f90710 100644 --- a/debian/clickhouse-client.install +++ b/debian/clickhouse-client.install @@ -1,6 +1,6 @@ -/usr/bin/clickhouse-client -/usr/bin/clickhouse-local -/usr/bin/clickhouse-compressor -/usr/bin/clickhouse-benchmark -/etc/clickhouse-client/config.xml -/usr/bin/clickhouse-extract-from-config +usr/bin/clickhouse-client +usr/bin/clickhouse-local +usr/bin/clickhouse-compressor +usr/bin/clickhouse-benchmark +etc/clickhouse-client/config.xml +usr/bin/clickhouse-extract-from-config diff --git a/debian/clickhouse-common-static.install b/debian/clickhouse-common-static.install new file mode 100644 index 00000000000..a2f5b69c964 --- /dev/null +++ b/debian/clickhouse-common-static.install @@ -0,0 +1,3 @@ +usr/bin/clickhouse +etc/security/limits.d/clickhouse.conf +usr/share/clickhouse/* diff --git a/debian/clickhouse-server-base.install b/debian/clickhouse-server-base.install deleted file mode 100644 index 573f34042bb..00000000000 --- a/debian/clickhouse-server-base.install +++ /dev/null @@ -1,11 +0,0 @@ -/usr/bin/clickhouse -/usr/bin/clickhouse-server -/usr/bin/clickhouse-clang -/usr/bin/clickhouse-lld -/usr/bin/clickhouse-copier -/usr/bin/clickhouse-report -/etc/systemd/system/clickhouse-server.service -/etc/init.d/clickhouse-server -/etc/cron.d/clickhouse-server -/usr/share/clickhouse/* -/etc/security/limits.d/clickhouse.conf diff --git a/debian/clickhouse-server-common.install b/debian/clickhouse-server-common.install deleted file mode 100644 index 7237e9914d5..00000000000 --- a/debian/clickhouse-server-common.install +++ /dev/null @@ -1,2 +0,0 @@ -etc/clickhouse-server/config.xml etc/clickhouse-server -etc/clickhouse-server/users.xml etc/clickhouse-server diff --git a/debian/clickhouse-server-metrika.links b/debian/clickhouse-server-metrika.links new file mode 100644 index 00000000000..037f5272b94 --- /dev/null +++ b/debian/clickhouse-server-metrika.links @@ -0,0 +1 @@ +usr/bin/clickhouse usr/bin/clickhouse-server diff --git a/debian/clickhouse-server-base.docs b/debian/clickhouse-server.docs similarity index 100% rename from debian/clickhouse-server-base.docs rename to debian/clickhouse-server.docs diff --git a/debian/clickhouse-server.install b/debian/clickhouse-server.install new file mode 100644 index 00000000000..f69969a6084 --- /dev/null +++ b/debian/clickhouse-server.install @@ -0,0 +1,8 @@ +usr/bin/clickhouse-server +usr/bin/clickhouse-clang +usr/bin/clickhouse-lld +usr/bin/clickhouse-copier +usr/bin/clickhouse-report +etc/clickhouse-server/config.xml +etc/clickhouse-server/users.xml +etc/systemd/system/clickhouse-server.service diff --git a/debian/clickhouse-server-base.postinst b/debian/clickhouse-server.postinst similarity index 100% rename from debian/clickhouse-server-base.postinst rename to debian/clickhouse-server.postinst diff --git a/debian/clickhouse-server-base.preinst b/debian/clickhouse-server.preinst similarity index 100% rename from debian/clickhouse-server-base.preinst rename to debian/clickhouse-server.preinst diff --git a/debian/clickhouse-server-base.prerm b/debian/clickhouse-server.prerm similarity index 100% rename from debian/clickhouse-server-base.prerm rename to debian/clickhouse-server.prerm diff --git a/debian/control b/debian/control index 5edac5f66a8..46b743636a5 100644 --- a/debian/control +++ b/debian/control @@ -13,11 +13,11 @@ Build-Depends: debhelper (>= 9), libreadline-dev, libssl-dev, unixodbc-dev -Standards-Version: 3.8.0 +Standards-Version: 3.9.8 Package: clickhouse-client Architecture: any -Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-server-base (= ${binary:Version}) +Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}) Replaces: clickhouse-compressor Conflicts: clickhouse-compressor Description: Client binary for clickhouse @@ -26,28 +26,37 @@ Description: Client binary for clickhouse . This package provides clickhouse-client , clickhouse-local and clickhouse-benchmark -Package: clickhouse-server-base +Package: clickhouse-common-static Architecture: any -Depends: ${shlibs:Depends}, ${misc:Depends}, adduser, tzdata +Depends: ${shlibs:Depends}, ${misc:Depends} +Replaces: clickhouse-server-base +Provides: clickhouse-server-base +Conflicts: clickhouse-server-base +Description: Common files for clickhouse + Yandex ClickHouse is a column-oriented database management system + that allows generating analytical data reports in real time. + . + This package provides common files for both clickhouse server and client + +Package: clickhouse-server +Architecture: any +Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-common-static (= ${binary:Version}), adduser, tzdata +Replaces: clickhouse-server-common +Provides: clickhouse-server-common +Conflicts: clickhouse-server-common Description: Server binary for clickhouse Yandex ClickHouse is a column-oriented database management system that allows generating analytical data reports in real time. . This package provides clickhouse common configuration files -Package: clickhouse-server-common -Architecture: any -Depends: ${shlibs:Depends}, ${misc:Depends}, clickhouse-server-base (= ${binary:Version}) -Description: clickhouse-server-common - Common configuration files for clickhouse-server-base package - Package: clickhouse-common-dbg Architecture: any Section: debug Priority: extra -Depends: ${misc:Depends}, clickhouse-server-base (= ${binary:Version}) -Description: debugging symbols for clickhouse-server-base - This package contains the debugging symbols for clickhouse-server-base. +Depends: ${misc:Depends}, clickhouse-common-static (= ${binary:Version}) +Description: debugging symbols for clickhouse-common-static + This package contains the debugging symbols for clickhouse-common. Package: clickhouse-test Priority: extra diff --git a/debian/rules b/debian/rules index c9d419b151b..fe53eac4643 100755 --- a/debian/rules +++ b/debian/rules @@ -66,32 +66,26 @@ override_dh_auto_test: override_dh_clean: rm -rf $(BUILDDIR) rm -rf $(DESTDIR) - rm -rf debian/copyright debian/clickhouse-server-common.docs debian/clickhouse-client.docs + rm -rf debian/copyright debian/clickhouse-client.docs debian/clickhouse-common-static.docs dh_clean override_dh_strip: - dh_strip -pclickhouse-server-base --dbg-package=clickhouse-common-dbg + dh_strip -pclickhouse-common-static --dbg-package=clickhouse-common-dbg override_dh_install: # Making docs cp LICENSE debian/copyright - ln -sf clickhouse-server-base.docs debian/clickhouse-client.docs - ln -sf clickhouse-server-base.docs debian/clickhouse-server-common.docs + ln -sf clickhouse-server.docs debian/clickhouse-client.docs + ln -sf clickhouse-server.docs debian/clickhouse-common-static.docs mkdir -p $(DESTDIR)/etc/security/limits.d cp debian/clickhouse.limits $(DESTDIR)/etc/security/limits.d/clickhouse.conf - # todo: remove after renaming package: - mkdir -p $(DESTDIR)/etc/init.d - cp debian/clickhouse-server.init $(DESTDIR)/etc/init.d/clickhouse-server # systemd compatibility mkdir -p $(DESTDIR)/etc/systemd/system/ cp debian/clickhouse-server.service $(DESTDIR)/etc/systemd/system/ - mkdir -p $(DESTDIR)/etc/cron.d - cp debian/clickhouse-server.cron.d $(DESTDIR)/etc/cron.d/clickhouse-server - # In case building clickhouse-server, adding to package binary of clang, ld and header files - for dynamic compilation. mkdir -p $(DESTDIR)/usr/share/clickhouse/headers diff --git a/debian/source/options b/debian/source/options index 9521b6828cc..51bf19d294a 100644 --- a/debian/source/options +++ b/debian/source/options @@ -1,6 +1,6 @@ tar-ignore -tar-ignore="build" -tar-ignore="build_*" +tar-ignore="build/*" +tar-ignore="build_*/*" tar-ignore="contrib/poco/openssl/*" tar-ignore="contrib/poco/gradle/*" tar-ignore="contrib/poco/Data/SQLite/*" diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 2d4f998eff1..20882fbeee3 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -8,7 +8,7 @@ RUN apt-get update && \ mkdir -p /etc/apt/sources.list.d && \ echo $repository | tee /etc/apt/sources.list.d/clickhouse.list && \ apt-get update && \ - apt-get install --allow-unauthenticated -y clickhouse-server-common=$version clickhouse-server-base=$version && \ + apt-get install --allow-unauthenticated -y clickhouse-server=$version && \ rm -rf /var/lib/apt/lists/* /var/cache/debconf && \ apt-get clean diff --git a/docs/en/getting_started/index.md b/docs/en/getting_started/index.md index d3e9ea03915..10ed75040d8 100755 --- a/docs/en/getting_started/index.md +++ b/docs/en/getting_started/index.md @@ -31,7 +31,7 @@ Then run: ```bash sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-get update -sudo apt-get install clickhouse-client clickhouse-server-common +sudo apt-get install clickhouse-client clickhouse-server ``` You can also download and install packages manually from here: diff --git a/docs/ru/getting_started/index.md b/docs/ru/getting_started/index.md index 3847663b3d5..2198ab2bc7d 100644 --- a/docs/ru/getting_started/index.md +++ b/docs/ru/getting_started/index.md @@ -31,7 +31,7 @@ deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ ```bash sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-get update -sudo apt-get install clickhouse-client clickhouse-server-common +sudo apt-get install clickhouse-client clickhouse-server ``` Также можно скачать и установить пакеты вручную, отсюда: . diff --git a/website/deprecated/reference_en.html b/website/deprecated/reference_en.html index 728c9622087..e6e4dee6227 100644 --- a/website/deprecated/reference_en.html +++ b/website/deprecated/reference_en.html @@ -439,7 +439,7 @@ Then run: %% sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-get update -sudo apt-get install clickhouse-client clickhouse-server-common +sudo apt-get install -y clickhouse-client clickhouse-server %% You can also download and install packages manually from here: @@ -709,7 +709,7 @@ echo 'DROP TABLE t' | POST 'http://localhost:8123/' For successful requests that don't return a data table, an empty response body is returned. -You can use compression when transmitting data. The compressed data has a non-standard format, and you will need to use a special compressor program to work with it (%%sudo apt-get install clickhouse-compressor%%). +You can use compression when transmitting data. The compressed data has a non-standard format, and you will need to use a special clickhouse-compressor program to work with it (%%sudo apt-get install clickhouse-utils%%). If you specified 'compress=1' in the URL, the server will compress the data it sends you. If you specified 'decompress=1' in the URL, the server will decompress the same data that you pass in the POST method. diff --git a/website/deprecated/reference_ru.html b/website/deprecated/reference_ru.html index c7b4126a167..2965054a737 100644 --- a/website/deprecated/reference_ru.html +++ b/website/deprecated/reference_ru.html @@ -449,7 +449,7 @@ deb http://repo.yandex.ru/clickhouse/trusty stable main %% sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-get update -sudo apt-get install clickhouse-client clickhouse-server-common +sudo apt-get install -y clickhouse-client clickhouse-server %% Также можно скачать и установить пакеты вручную, отсюда: @@ -725,7 +725,7 @@ echo 'DROP TABLE t' | POST 'http://localhost:8123/' Для запросов, которые не возвращают таблицу с данными, в случае успеха, выдаётся пустое тело ответа. -Вы можете использовать сжатие при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу compressor (%%sudo apt-get install clickhouse-compressor%%). +Вы можете использовать сжатие при передаче данных. Формат сжатых данных нестандартный, и вам придётся использовать для работы с ним специальную программу clickhouse-compressor (%%sudo apt-get install clickhouse-utils%%). Если вы указали в URL compress=1, то сервер будет сжимать отправляемые вам данные. Если вы указали в URL decompress=1, то сервер будет разжимать те данные, которые вы передаёте ему POST-ом. diff --git a/website/index.html b/website/index.html index e315b78199d..78b89d3b07b 100644 --- a/website/index.html +++ b/website/index.html @@ -393,7 +393,7 @@ sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 # optional sudo apt-add-repository "deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" sudo apt-get update -sudo apt-get install clickhouse-server-common clickhouse-client -y +sudo apt-get install -y clickhouse-server clickhouse-client sudo service clickhouse-server start clickhouse-client diff --git a/website/tutorial.html b/website/tutorial.html index 0472bef268d..558d9a0d0fe 100644 --- a/website/tutorial.html +++ b/website/tutorial.html @@ -51,7 +51,7 @@

clickhouse-client package contains clickhouse-client application — - interactive ClickHouse client. clickhouse-server-base contains a clickhouse-server binary file. clickhouse-server-common + interactive ClickHouse client. clickhouse-common contains a clickhouse-server binary file. clickhouse-server — contains config files for the clickhouse-server.

Server config files are located in /etc/clickhouse-server/. Before getting to work please notice the path From a1aecc5d96796f84e11764232ba7352e319a1c01 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 1 Apr 2018 20:37:09 +0300 Subject: [PATCH 62/70] Fixed error after merge [#CLICKHOUSE-2] --- dbms/src/Server/ClusterCopier.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index 1c475a6a604..61fe2234124 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -1262,10 +1262,10 @@ protected: } /// Remove the locking node - zkutil::Ops ops; - ops.emplace_back(new zkutil::Op::Remove(dirt_cleaner_path, -1)); - ops.emplace_back(new zkutil::Op::Remove(is_dirty_flag_path, -1)); - zookeeper->multi(ops); + zkutil::Requests requests; + requests.emplace_back(zkutil::makeRemoveRequest(dirt_cleaner_path, -1)); + requests.emplace_back(zkutil::makeRemoveRequest(is_dirty_flag_path, -1)); + zookeeper->multi(requests); LOG_INFO(log, "Partition " << task_partition.name << " was dropped on cluster " << task_table.cluster_push_name); return true; From 3fe743e2bd50a58e86e13774c7141d22d0283695 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 1 Apr 2018 22:29:08 +0300 Subject: [PATCH 63/70] Fixed error after merge [#CLICKHOUSE-2] --- dbms/src/Storages/MergeTree/MergeTreeReader.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 6814497e058..ef802fc633c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -482,7 +482,7 @@ void MergeTreeReader::fillMissingColumns(Block & res, bool & should_reorder, boo { /// We must turn a constant column into a full column because the interpreter could infer that it is constant everywhere /// but in some blocks (from other parts) it can be a full column. - column_to_add.column = column_to_add.type->createColumnConstWithDefaultValue(rows)->convertToFullColumnIfConst(); + column_to_add.column = column_to_add.type->createColumnConstWithDefaultValue(res.rows())->convertToFullColumnIfConst(); } res.insert(std::move(column_to_add)); @@ -521,7 +521,7 @@ void MergeTreeReader::evaluateMissingDefaults(Block & res) { try { - DB::evaluateMissingDefaults(res, columns, storage.column_defaults, storage.context); + DB::evaluateMissingDefaults(res, columns, storage.getColumns().defaults, storage.context); } catch (Exception & e) { From 23810748483c0c913d496ff650d9e039b7fad8d3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 1 Apr 2018 23:13:18 +0300 Subject: [PATCH 64/70] Updated zstd to v1.3.4 [#CLICKHOUSE-2] --- contrib/zstd | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/zstd b/contrib/zstd index f4340f46b23..255597502c3 160000 --- a/contrib/zstd +++ b/contrib/zstd @@ -1 +1 @@ -Subproject commit f4340f46b2387bc8de7d5320c0b83bb1499933ad +Subproject commit 255597502c3a4ef150abc964e376d4202a8c2929 From 66857317d4cecc89df1144cfa32fcf1db36cb5b1 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 1 Apr 2018 23:29:48 +0300 Subject: [PATCH 65/70] Removed DOS style newlines from CONTRIBUTING.md [#CLICKHOUSE-2] --- CONTRIBUTING.md | 64 ++++++++++++++++++++++++------------------------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index ec8f5735957..761afd0f657 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,32 +1,32 @@ -# Contributing to ClickHouse - -## Technical info -Developer guide for writing code for ClickHouse is published on official website alongside the usage and operations documentation: -https://clickhouse.yandex/docs/en/development/index.html - -## Legal info - -In order for us (YANDEX LLC) to accept patches and other contributions from you, you will have to adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here: -1) https://yandex.ru/legal/cla/?lang=en (in English) and -2) https://yandex.ru/legal/cla/?lang=ru (in Russian). - -By adopting the CLA, you state the following: - -* You obviously wish and are willingly licensing your contributions to us for our open source projects under the terms of the CLA, -* You has read the terms and conditions of the CLA and agree with them in full, -* You are legally able to provide and license your contributions as stated, -* We may use your contributions for our open source projects and for any other our project too, -* We rely on your assurances concerning the rights of third parties in relation to your contributes. - -If you agree with these principles, please read and adopt our CLA. By providing us your contributions, you hereby declare that you has already read and adopt our CLA, and we may freely merge your contributions with our corresponding open source project and use it in further in accordance with terms and conditions of the CLA. - -If you have already adopted terms and conditions of the CLA, you are able to provide your contributes. When you submit your pull request, please add the following information into it: - -``` -I hereby agree to the terms of the CLA available at: [link]. -``` - -Replace the bracketed text as follows: -* [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian). - -It is enough to provide us such notification at once. +# Contributing to ClickHouse + +## Technical info +Developer guide for writing code for ClickHouse is published on official website alongside the usage and operations documentation: +https://clickhouse.yandex/docs/en/development/index.html + +## Legal info + +In order for us (YANDEX LLC) to accept patches and other contributions from you, you will have to adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here: +1) https://yandex.ru/legal/cla/?lang=en (in English) and +2) https://yandex.ru/legal/cla/?lang=ru (in Russian). + +By adopting the CLA, you state the following: + +* You obviously wish and are willingly licensing your contributions to us for our open source projects under the terms of the CLA, +* You has read the terms and conditions of the CLA and agree with them in full, +* You are legally able to provide and license your contributions as stated, +* We may use your contributions for our open source projects and for any other our project too, +* We rely on your assurances concerning the rights of third parties in relation to your contributes. + +If you agree with these principles, please read and adopt our CLA. By providing us your contributions, you hereby declare that you has already read and adopt our CLA, and we may freely merge your contributions with our corresponding open source project and use it in further in accordance with terms and conditions of the CLA. + +If you have already adopted terms and conditions of the CLA, you are able to provide your contributes. When you submit your pull request, please add the following information into it: + +``` +I hereby agree to the terms of the CLA available at: [link]. +``` + +Replace the bracketed text as follows: +* [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian). + +It is enough to provide us such notification at once. From 99a2d821423fe69dbf3af2a1abe37536924c61bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 1 Apr 2018 23:31:15 +0300 Subject: [PATCH 66/70] Removed trailing whitespaces from CONTRIBUTING.md [#CLICKHOUSE-2] --- CONTRIBUTING.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 761afd0f657..b40c7be38cc 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,4 +1,4 @@ -# Contributing to ClickHouse +# Contributing to ClickHouse ## Technical info Developer guide for writing code for ClickHouse is published on official website alongside the usage and operations documentation: @@ -7,7 +7,7 @@ https://clickhouse.yandex/docs/en/development/index.html ## Legal info In order for us (YANDEX LLC) to accept patches and other contributions from you, you will have to adopt our Yandex Contributor License Agreement (the "**CLA**"). The current version of the CLA you may find here: -1) https://yandex.ru/legal/cla/?lang=en (in English) and +1) https://yandex.ru/legal/cla/?lang=en (in English) and 2) https://yandex.ru/legal/cla/?lang=ru (in Russian). By adopting the CLA, you state the following: @@ -29,4 +29,4 @@ I hereby agree to the terms of the CLA available at: [link]. Replace the bracketed text as follows: * [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian). -It is enough to provide us such notification at once. +It is enough to provide us such notification at once. From 36d093513161bf05541d02947d741e663580e5ea Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 1 Apr 2018 23:32:24 +0300 Subject: [PATCH 67/70] Added a notice to CONTRIBUTING.md [#CLICKHOUSE-2] --- CONTRIBUTING.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index b40c7be38cc..56b4c993981 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -30,3 +30,5 @@ Replace the bracketed text as follows: * [link] is the link at the current version of the CLA (you may add here a link https://yandex.ru/legal/cla/?lang=en (in English) or a link https://yandex.ru/legal/cla/?lang=ru (in Russian). It is enough to provide us such notification at once. + +If you don't agree with the CLA, you still can open a pull request to provide your contributions. From ad137994f96026440cd6a50633ba4da9815c9dd2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 2 Apr 2018 02:16:41 +0300 Subject: [PATCH 68/70] Modifications after removing libzookeeper [#CLICKHOUSE-2] --- dbms/tests/instructions/sanitizers.md | 2 +- dbms/tests/instructions/tsan_suppressions | 2 -- 2 files changed, 1 insertion(+), 3 deletions(-) delete mode 100644 dbms/tests/instructions/tsan_suppressions diff --git a/dbms/tests/instructions/sanitizers.md b/dbms/tests/instructions/sanitizers.md index feb35257b93..4eb155892ff 100644 --- a/dbms/tests/instructions/sanitizers.md +++ b/dbms/tests/instructions/sanitizers.md @@ -50,7 +50,7 @@ scp ./dbms/src/Server/clickhouse yourserver:~/clickhouse-tsan ## Start ClickHouse and run tests ``` -sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1 suppressions=tsan_suppressions' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml +sudo -u clickhouse TSAN_OPTIONS='halt_on_error=1' ./clickhouse-tsan server --config /etc/clickhouse-server/config.xml ``` diff --git a/dbms/tests/instructions/tsan_suppressions b/dbms/tests/instructions/tsan_suppressions deleted file mode 100644 index 1bc366e12be..00000000000 --- a/dbms/tests/instructions/tsan_suppressions +++ /dev/null @@ -1,2 +0,0 @@ -# ZooKeeper C library is a trash: -race:contrib/zookeeper From 440b0f33c78830201c6fd6eb73e1ddee40f745b3 Mon Sep 17 00:00:00 2001 From: proller Date: Mon, 2 Apr 2018 15:31:04 +0300 Subject: [PATCH 69/70] Compile fixes --- dbms/src/Storages/Distributed/DistributedBlockOutputStream.h | 1 + dbms/src/Storages/MergeTree/MergeTreeRangeReader.h | 2 ++ utils/check_include.sh | 1 + 3 files changed, 4 insertions(+) diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h index fe9e765021b..b08a21bbc07 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -10,6 +10,7 @@ #include #include #include +#include namespace Poco { diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index afba6715c0a..21267b517e9 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -1,6 +1,8 @@ #pragma once #include #include +#include +#include namespace DB { diff --git a/utils/check_include.sh b/utils/check_include.sh index 76e5feca798..474f08e1a44 100755 --- a/utils/check_include.sh +++ b/utils/check_include.sh @@ -36,6 +36,7 @@ inc="-I. \ -I./libs/libpocoext/include \ -I./libs/libzkutil/include \ -I./libs/libdaemon/include \ +-I./libs/libconsistent-hashing \ -I./dbms/src \ -I${BUILD_DIR}/dbms/src" From 62e6a1bde6f44f61395e3d6e0e4f1944a9f43d88 Mon Sep 17 00:00:00 2001 From: KochetovNicolai Date: Mon, 2 Apr 2018 17:08:05 +0300 Subject: [PATCH 70/70] Update MergeTreeBaseBlockInputStream.cpp Check is block is not empty before removing prewhere column --- dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index 0bc13cb0c10..9f8525d2092 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -168,7 +168,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() task->size_predictor->update(read_result.block); } - if (prewhere_actions && !task->remove_prewhere_column) + if (read_result.block && prewhere_actions && !task->remove_prewhere_column) { /// Convert const column to full here because it's cheaper to filter const column than full. auto & column = read_result.block.getByName(prewhere_column_name);