From d388918c607f166c49a533375279053dd9f5d733 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 6 Nov 2019 22:39:52 +0300 Subject: [PATCH 01/77] Join.cpp refactoring --- dbms/src/Interpreters/Join.cpp | 177 +++++++++++++++++---------------- dbms/src/Interpreters/Join.h | 10 +- 2 files changed, 92 insertions(+), 95 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 12f319fdf7a..8e510afb54e 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -582,7 +582,17 @@ public: const Block & block_with_columns_to_add, const Block & block, const Block & saved_block_sample, - const ColumnsWithTypeAndName & extras) + const ColumnsWithTypeAndName & extras, + const Join & join_, + const ColumnRawPtrs & key_columns_, + const Sizes & key_sizes_, + ConstNullMapPtr null_map_) + : join(join_) + , key_columns(key_columns_) + , key_sizes(key_sizes_) + , null_map(null_map_) + , rows_to_add(block.rows()) + , filter(rows_to_add, 0) { size_t num_columns_to_add = sample_block_with_columns_to_add.columns(); @@ -615,21 +625,39 @@ public: void appendFromBlock(const Block & block, size_t row_num) { + applyLazyDefaults(); for (size_t j = 0; j < right_indexes.size(); ++j) columns[j]->insertFrom(*block.getByPosition(right_indexes[j]).column, row_num); } - void appendDefaultRow() { - for (size_t j = 0; j < right_indexes.size(); ++j) - columns[j]->insertDefault(); + ++lazy_defaults_count; } + void applyLazyDefaults() + { + if (lazy_defaults_count) + { + for (size_t j = 0; j < right_indexes.size(); ++j) + columns[j]->insertManyDefaults(lazy_defaults_count); + lazy_defaults_count = 0; + } + } + + const Join & join; + const ColumnRawPtrs & key_columns; + const Sizes & key_sizes; + ConstNullMapPtr null_map; + size_t rows_to_add; + IColumn::Filter filter; + std::unique_ptr offsets_to_replicate; + private: TypeAndNames type_name; MutableColumns columns; std::vector right_indexes; + size_t lazy_defaults_count = 0; void addColumn(const ColumnWithTypeAndName & src_column) { @@ -639,21 +667,13 @@ private: } }; -template -void addFoundRow(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset [[maybe_unused]]) +template +void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset) { - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Any) + for (auto it = mapped.begin(); it.ok(); ++it) { - added.appendFromBlock(*mapped.block, mapped.row_num); - } - - if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) - { - for (auto it = mapped.begin(); it.ok(); ++it) - { - added.appendFromBlock(*it->block, it->row_num); - ++current_offset; - } + added.appendFromBlock(*it->block, it->row_num); + ++current_offset; } }; @@ -671,27 +691,26 @@ void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & cur /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). template -std::unique_ptr NO_INLINE joinRightIndexedColumns( - const Join & join, const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, - AddedColumns & added_columns, ConstNullMapPtr null_map, IColumn::Filter & filter) +void NO_INLINE joinRightIndexedColumns(const Map & map, AddedColumns & added_columns) { - std::unique_ptr offsets_to_replicate; + size_t rows = added_columns.rows_to_add; if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) - offsets_to_replicate = std::make_unique(rows); + added_columns.offsets_to_replicate = std::make_unique(rows); + IColumn::Filter & filter = added_columns.filter; Arena pool; const IColumn * asof_column [[maybe_unused]] = nullptr; if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) - asof_column = extractAsofColumn(key_columns); - auto key_getter = createKeyGetter(key_columns, key_sizes); + asof_column = extractAsofColumn(added_columns.key_columns); + auto key_getter = createKeyGetter(added_columns.key_columns, added_columns.key_sizes); IColumn::Offset current_offset = 0; for (size_t i = 0; i < rows; ++i) { - if (_has_null_map && (*null_map)[i]) + if (_has_null_map && (*added_columns.null_map)[i]) { addNotFoundRow<_add_missing>(added_columns, current_offset); } @@ -702,23 +721,31 @@ std::unique_ptr NO_INLINE joinRightIndexedColumns( if (find_result.isFound()) { auto & mapped = find_result.getMapped(); + filter[i] = 1; if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) { + const Join & join = added_columns.join; if (const RowRef * found = mapped.findAsof(join.getAsofType(), join.getAsofInequality(), asof_column, i)) { - filter[i] = 1; mapped.setUsed(); added_columns.appendFromBlock(*found->block, found->row_num); } else + { + filter[i] = 0; addNotFoundRow<_add_missing>(added_columns, current_offset); + } + } + else if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) + { + mapped.setUsed(); + addFoundRowAll(mapped, added_columns, current_offset); } else { - filter[i] = 1; mapped.setUsed(); - addFoundRow(mapped, added_columns, current_offset); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); } } else @@ -726,44 +753,33 @@ std::unique_ptr NO_INLINE joinRightIndexedColumns( } if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) - (*offsets_to_replicate)[i] = current_offset; + (*added_columns.offsets_to_replicate)[i] = current_offset; } - return offsets_to_replicate; + added_columns.applyLazyDefaults(); } template -IColumn::Filter joinRightColumns( - const Join & join, const Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, - AddedColumns & added_columns, ConstNullMapPtr null_map, std::unique_ptr & offsets_to_replicate) +void joinRightColumns(const Map & map, AddedColumns & added_columns) { constexpr bool left_or_full = static_in_v; - IColumn::Filter filter(rows, 0); - - if (null_map) - offsets_to_replicate = joinRightIndexedColumns( - join, map, rows, key_columns, key_sizes, added_columns, null_map, filter); + if (added_columns.null_map) + joinRightIndexedColumns(map, added_columns); else - offsets_to_replicate = joinRightIndexedColumns( - join, map, rows, key_columns, key_sizes, added_columns, null_map, filter); - - return filter; + joinRightIndexedColumns(map, added_columns); } template -IColumn::Filter switchJoinRightColumns( - Join::Type type, const Join & join, - const Maps & maps_, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, - AddedColumns & added_columns, ConstNullMapPtr null_map, - std::unique_ptr & offsets_to_replicate) +void switchJoinRightColumns(const Maps & maps_, AddedColumns & added_columns, Join::Type type) { switch (type) { #define M(TYPE) \ case Join::Type::TYPE: \ - return joinRightColumns>::Type>(\ - join, *maps_.TYPE, rows, key_columns, key_sizes, added_columns, null_map, offsets_to_replicate); + joinRightColumns>::Type>(\ + *maps_.TYPE, added_columns); \ + break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -775,12 +791,7 @@ IColumn::Filter switchJoinRightColumns( } /// nameless -template -void Join::joinBlockImpl( - Block & block, - const Names & key_names_left, - const Block & block_with_columns_to_add, - const Maps & maps_) const +void Join::joinBlockImpl(Block & block, const Names & key_names_left, const Block & block_with_columns_to_add) const { /// Rare case, when keys are constant. To avoid code bloat, simply materialize them. Columns materialized_columns; @@ -796,8 +807,8 @@ void Join::joinBlockImpl( * Because if they are constants, then in the "not joined" rows, they may have different values * - default values, which can differ from the values of these constants. */ - constexpr bool right_or_full = static_in_v; - if constexpr (right_or_full) + bool right_or_full = isRightOrFull(kind); + if (right_or_full) { materializeBlockInplace(block); @@ -811,25 +822,31 @@ void Join::joinBlockImpl( * For ASOF, the last column is used as the ASOF column */ ColumnsWithTypeAndName extras; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + if (strictness == ASTTableJoin::Strictness::Asof) extras.push_back(right_table_keys.getByName(key_names_right.back())); - AddedColumns added(sample_block_with_columns_to_add, block_with_columns_to_add, block, saved_block_sample, extras); - std::unique_ptr offsets_to_replicate; + AddedColumns added_columns(sample_block_with_columns_to_add, block_with_columns_to_add, block, saved_block_sample, + extras, *this, key_columns, key_sizes, null_map); - IColumn::Filter row_filter = switchJoinRightColumns( - type, *this, maps_, block.rows(), key_columns, key_sizes, added, null_map, offsets_to_replicate); + if (!joinDispatch(kind, strictness, maps, + [&](auto kind_, auto strictness_, auto & maps_) + { + switchJoinRightColumns(maps_, added_columns, type); + })) + throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR); - for (size_t i = 0; i < added.size(); ++i) - block.insert(added.moveColumn(i)); + IColumn::Filter & row_filter = added_columns.filter; + + for (size_t i = 0; i < added_columns.size(); ++i) + block.insert(added_columns.moveColumn(i)); /// Filter & insert missing rows - constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; - constexpr bool inner_or_right = static_in_v; + bool is_all_join = (strictness == ASTTableJoin::Strictness::All); + bool inner_or_right = isInner(kind) || isRight(kind); std::vector right_keys_to_replicate [[maybe_unused]]; - if constexpr (!is_all_join && inner_or_right) + if (!is_all_join && inner_or_right) { /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) @@ -866,13 +883,14 @@ void Join::joinBlockImpl( ColumnPtr thin_column = filterWithBlanks(col.column, filter); block.insert(correctNullability({thin_column, col.type, right_key.name}, is_nullable, null_map_filter)); - if constexpr (is_all_join) + if (is_all_join) right_keys_to_replicate.push_back(block.getPositionByName(right_key.name)); } } - if constexpr (is_all_join) + if (is_all_join) { + std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; if (!offsets_to_replicate) throw Exception("No data to filter columns", ErrorCodes::LOGICAL_ERROR); @@ -961,14 +979,6 @@ DataTypePtr Join::joinGetReturnType(const String & column_name) const } -template -void Join::joinGetImpl(Block & block, const String & column_name, const Maps & maps_) const -{ - joinBlockImpl( - block, {block.getByPosition(0).name}, {sample_block_with_columns_to_add.getByName(column_name)}, maps_); -} - - // TODO: support composite key // TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All @@ -983,7 +993,7 @@ void Join::joinGet(Block & block, const String & column_name) const if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) { - joinGetImpl(block, column_name, std::get(maps)); + joinBlockImpl(block, {block.getByPosition(0).name}, {sample_block_with_columns_to_add.getByName(column_name)}); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::LOGICAL_ERROR); @@ -997,17 +1007,10 @@ void Join::joinBlock(Block & block) const Names & key_names_left = table_join->keyNamesLeft(); JoinCommon::checkTypesOfKeys(block, key_names_left, right_table_keys, key_names_right); - if (joinDispatch(kind, strictness, maps, [&](auto kind_, auto strictness_, auto & map) - { - joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); - })) - { - /// Joined - } - else if (kind == ASTTableJoin::Kind::Cross) + if (kind == ASTTableJoin::Kind::Cross) joinBlockImplCross(block); else - throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR); + joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add); } diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 3553680b879..2e5b57cf89d 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -347,17 +347,11 @@ private: void initRightBlockStructure(); void initRequiredRightKeys(); - template - void joinBlockImpl( - Block & block, + void joinBlockImpl(Block & block, const Names & key_names_left, - const Block & block_with_columns_to_add, - const Maps & maps) const; + const Block & block_with_columns_to_add) const; void joinBlockImplCross(Block & block) const; - - template - void joinGetImpl(Block & block, const String & column_name, const Maps & maps) const; }; } From 252fab54a21fefafdde3c5d6cea98000dc75f359 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 6 Nov 2019 23:25:27 +0300 Subject: [PATCH 02/77] less specializations --- dbms/src/Interpreters/Join.cpp | 33 +++++++++++++++++---------------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 8e510afb54e..a341089efd2 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -690,8 +690,8 @@ void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & cur /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). -template -void NO_INLINE joinRightIndexedColumns(const Map & map, AddedColumns & added_columns) +template +void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) { size_t rows = added_columns.rows_to_add; if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) @@ -759,26 +759,27 @@ void NO_INLINE joinRightIndexedColumns(const Map & map, AddedColumns & added_col added_columns.applyLazyDefaults(); } -template -void joinRightColumns(const Map & map, AddedColumns & added_columns) +template +void joinRightColumnsSwitchNullability(const Map & map, AddedColumns & added_columns) +{ + if (added_columns.null_map) + joinRightColumns(map, added_columns); + else + joinRightColumns(map, added_columns); +} + +template +void switchJoinRightColumns(const Maps & maps_, AddedColumns & added_columns, Join::Type type) { constexpr bool left_or_full = static_in_v; - if (added_columns.null_map) - joinRightIndexedColumns(map, added_columns); - else - joinRightIndexedColumns(map, added_columns); -} - -template -void switchJoinRightColumns(const Maps & maps_, AddedColumns & added_columns, Join::Type type) -{ switch (type) { #define M(TYPE) \ case Join::Type::TYPE: \ - joinRightColumns>::Type>(\ - *maps_.TYPE, added_columns); \ + joinRightColumnsSwitchNullability>::Type>(\ + *maps_.TYPE, added_columns);\ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -831,7 +832,7 @@ void Join::joinBlockImpl(Block & block, const Names & key_names_left, const Bloc if (!joinDispatch(kind, strictness, maps, [&](auto kind_, auto strictness_, auto & maps_) { - switchJoinRightColumns(maps_, added_columns, type); + switchJoinRightColumns(maps_, added_columns, type); })) throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR); From 2cafe933ea24d203d59671e5d654b79da092e89a Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 7 Nov 2019 15:42:39 +0300 Subject: [PATCH 03/77] revert some changes --- dbms/src/Interpreters/Join.cpp | 62 +++++++++++++++++++++------------- dbms/src/Interpreters/Join.h | 10 ++++-- 2 files changed, 46 insertions(+), 26 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index a341089efd2..797f43546bd 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -759,25 +759,25 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) added_columns.applyLazyDefaults(); } -template +template void joinRightColumnsSwitchNullability(const Map & map, AddedColumns & added_columns) { + constexpr bool left_or_full = static_in_v; + if (added_columns.null_map) joinRightColumns(map, added_columns); else joinRightColumns(map, added_columns); } -template +template void switchJoinRightColumns(const Maps & maps_, AddedColumns & added_columns, Join::Type type) { - constexpr bool left_or_full = static_in_v; - switch (type) { #define M(TYPE) \ case Join::Type::TYPE: \ - joinRightColumnsSwitchNullability>::Type>(\ *maps_.TYPE, added_columns);\ break; @@ -792,7 +792,12 @@ void switchJoinRightColumns(const Maps & maps_, AddedColumns & added_columns, Jo } /// nameless -void Join::joinBlockImpl(Block & block, const Names & key_names_left, const Block & block_with_columns_to_add) const +template +void Join::joinBlockImpl( + Block & block, + const Names & key_names_left, + const Block & block_with_columns_to_add, + const Maps & maps_) const { /// Rare case, when keys are constant. To avoid code bloat, simply materialize them. Columns materialized_columns; @@ -808,8 +813,8 @@ void Join::joinBlockImpl(Block & block, const Names & key_names_left, const Bloc * Because if they are constants, then in the "not joined" rows, they may have different values * - default values, which can differ from the values of these constants. */ - bool right_or_full = isRightOrFull(kind); - if (right_or_full) + constexpr bool right_or_full = static_in_v; + if constexpr (right_or_full) { materializeBlockInplace(block); @@ -823,31 +828,25 @@ void Join::joinBlockImpl(Block & block, const Names & key_names_left, const Bloc * For ASOF, the last column is used as the ASOF column */ ColumnsWithTypeAndName extras; - if (strictness == ASTTableJoin::Strictness::Asof) + if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) extras.push_back(right_table_keys.getByName(key_names_right.back())); AddedColumns added_columns(sample_block_with_columns_to_add, block_with_columns_to_add, block, saved_block_sample, extras, *this, key_columns, key_sizes, null_map); - if (!joinDispatch(kind, strictness, maps, - [&](auto kind_, auto strictness_, auto & maps_) - { - switchJoinRightColumns(maps_, added_columns, type); - })) - throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR); - + switchJoinRightColumns(maps_, added_columns, type); IColumn::Filter & row_filter = added_columns.filter; for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); /// Filter & insert missing rows - bool is_all_join = (strictness == ASTTableJoin::Strictness::All); - bool inner_or_right = isInner(kind) || isRight(kind); + constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; + constexpr bool inner_or_right = static_in_v; std::vector right_keys_to_replicate [[maybe_unused]]; - if (!is_all_join && inner_or_right) + if constexpr (!is_all_join && inner_or_right) { /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) @@ -884,12 +883,12 @@ void Join::joinBlockImpl(Block & block, const Names & key_names_left, const Bloc ColumnPtr thin_column = filterWithBlanks(col.column, filter); block.insert(correctNullability({thin_column, col.type, right_key.name}, is_nullable, null_map_filter)); - if (is_all_join) + if constexpr (is_all_join) right_keys_to_replicate.push_back(block.getPositionByName(right_key.name)); } } - if (is_all_join) + if constexpr (is_all_join) { std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; if (!offsets_to_replicate) @@ -980,6 +979,14 @@ DataTypePtr Join::joinGetReturnType(const String & column_name) const } +template +void Join::joinGetImpl(Block & block, const String & column_name, const Maps & maps_) const +{ + joinBlockImpl( + block, {block.getByPosition(0).name}, {sample_block_with_columns_to_add.getByName(column_name)}, maps_); +} + + // TODO: support composite key // TODO: return multiple columns as named tuple // TODO: return array of values when strictness == ASTTableJoin::Strictness::All @@ -994,7 +1001,7 @@ void Join::joinGet(Block & block, const String & column_name) const if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) { - joinBlockImpl(block, {block.getByPosition(0).name}, {sample_block_with_columns_to_add.getByName(column_name)}); + joinGetImpl(block, column_name, std::get(maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::LOGICAL_ERROR); @@ -1008,10 +1015,17 @@ void Join::joinBlock(Block & block) const Names & key_names_left = table_join->keyNamesLeft(); JoinCommon::checkTypesOfKeys(block, key_names_left, right_table_keys, key_names_right); - if (kind == ASTTableJoin::Kind::Cross) + if (joinDispatch(kind, strictness, maps, [&](auto kind_, auto strictness_, auto & map) + { + joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add, map); + })) + { + /// Joined + } + else if (kind == ASTTableJoin::Kind::Cross) joinBlockImplCross(block); else - joinBlockImpl(block, key_names_left, sample_block_with_columns_to_add); + throw Exception("Logical error: unknown combination of JOIN", ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 2e5b57cf89d..3553680b879 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -347,11 +347,17 @@ private: void initRightBlockStructure(); void initRequiredRightKeys(); - void joinBlockImpl(Block & block, + template + void joinBlockImpl( + Block & block, const Names & key_names_left, - const Block & block_with_columns_to_add) const; + const Block & block_with_columns_to_add, + const Maps & maps) const; void joinBlockImplCross(Block & block) const; + + template + void joinGetImpl(Block & block, const String & column_name, const Maps & maps) const; }; } From b8d464aa03c22914b29d6ffde46f071b4b425b43 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 8 Nov 2019 00:32:44 +0300 Subject: [PATCH 04/77] new consistent ANY JOIN (LEFT, INNER, RIGHT) --- dbms/src/Interpreters/Join.cpp | 116 +++++++++++++----- dbms/src/Interpreters/Join.h | 29 +++-- dbms/src/Interpreters/SyntaxAnalyzer.cpp | 17 ++- dbms/src/Interpreters/joinDispatch.h | 61 +++------ dbms/src/Parsers/ASTTablesInSelectQuery.cpp | 1 + dbms/src/Parsers/ASTTablesInSelectQuery.h | 3 +- dbms/src/Storages/StorageJoin.cpp | 9 +- .../0_stateless/01031_new_any_join.reference | 32 +++++ .../0_stateless/01031_new_any_join.sql | 32 +++++ 9 files changed, 208 insertions(+), 92 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01031_new_any_join.reference create mode 100644 dbms/tests/queries/0_stateless/01031_new_any_join.sql diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 797f43546bd..4dffee8aa41 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -367,7 +367,7 @@ namespace }; template - struct Inserter + struct Inserter { static ALWAYS_INLINE void insert(const Join & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { @@ -427,7 +427,16 @@ namespace if (has_null_map && (*null_map)[i]) continue; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + if constexpr (STRICTNESS == ASTTableJoin::Strictness::Any) + { + constexpr bool mapped_one = std::is_same_v || + std::is_same_v; + if constexpr (mapped_one) + Inserter::insert(join, map, key_getter, stored_block, i, pool); + else + Inserter::insert(join, map, key_getter, stored_block, i, pool); + } + else if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) Inserter::insert(join, map, key_getter, stored_block, i, pool, asof_column); else Inserter::insert(join, map, key_getter, stored_block, i, pool); @@ -690,18 +699,25 @@ void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & cur /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). -template +template void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) { + constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; + constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; + constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; + constexpr bool left_or_full = static_in_v; + constexpr bool right = KIND == ASTTableJoin::Kind::Right; + + constexpr bool _add_missing = left_or_full; + constexpr bool need_replication = is_all_join || (is_any_join && right); + size_t rows = added_columns.rows_to_add; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) - added_columns.offsets_to_replicate = std::make_unique(rows); IColumn::Filter & filter = added_columns.filter; Arena pool; const IColumn * asof_column [[maybe_unused]] = nullptr; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + if constexpr (is_asof_join) asof_column = extractAsofColumn(added_columns.key_columns); auto key_getter = createKeyGetter(added_columns.key_columns, added_columns.key_sizes); @@ -721,13 +737,13 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) if (find_result.isFound()) { auto & mapped = find_result.getMapped(); - filter[i] = 1; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + if constexpr (is_asof_join) { const Join & join = added_columns.join; if (const RowRef * found = mapped.findAsof(join.getAsofType(), join.getAsofInequality(), asof_column, i)) { + filter[i] = 1; mapped.setUsed(); added_columns.appendFromBlock(*found->block, found->row_num); } @@ -737,13 +753,37 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) addNotFoundRow<_add_missing>(added_columns, current_offset); } } - else if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) + else if constexpr (is_all_join) { + filter[i] = 1; mapped.setUsed(); addFoundRowAll(mapped, added_columns, current_offset); } - else + else if constexpr (is_any_join && right) { + /// Use first appered left key + it needs left columns replication + if (mapped.setUsedOnce()) + { + filter[i] = 1; + addFoundRowAll(mapped, added_columns, current_offset); + } + } + else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Inner) + { + /// Use first appered left key only + if (mapped.setUsedOnce()) + { + filter[i] = 1; + added_columns.appendFromBlock(*mapped.block, mapped.row_num); + } + } + else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Full) + { + /// TODO + } + else /// ANY LEFT + old ANY (RightAny) + { + filter[i] = 1; mapped.setUsed(); added_columns.appendFromBlock(*mapped.block, mapped.row_num); } @@ -752,7 +792,7 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) addNotFoundRow<_add_missing>(added_columns, current_offset); } - if constexpr (STRICTNESS == ASTTableJoin::Strictness::All) + if constexpr (need_replication) (*added_columns.offsets_to_replicate)[i] = current_offset; } @@ -762,12 +802,10 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) template void joinRightColumnsSwitchNullability(const Map & map, AddedColumns & added_columns) { - constexpr bool left_or_full = static_in_v; - if (added_columns.null_map) - joinRightColumns(map, added_columns); + joinRightColumns(map, added_columns); else - joinRightColumns(map, added_columns); + joinRightColumns(map, added_columns); } template @@ -799,6 +837,16 @@ void Join::joinBlockImpl( const Block & block_with_columns_to_add, const Maps & maps_) const { + constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; + constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; + constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; + constexpr bool right = KIND == ASTTableJoin::Kind::Right; + constexpr bool inner_or_right = static_in_v; + constexpr bool right_or_full = static_in_v; + + constexpr bool need_filter = (!is_all_join && inner_or_right) && !(is_any_join && right); + constexpr bool need_replication = is_all_join || (is_any_join && right); + /// Rare case, when keys are constant. To avoid code bloat, simply materialize them. Columns materialized_columns; ColumnRawPtrs key_columns = JoinCommon::temporaryMaterializeColumns(block, key_names_left, materialized_columns); @@ -813,7 +861,6 @@ void Join::joinBlockImpl( * Because if they are constants, then in the "not joined" rows, they may have different values * - default values, which can differ from the values of these constants. */ - constexpr bool right_or_full = static_in_v; if constexpr (right_or_full) { materializeBlockInplace(block); @@ -828,25 +875,24 @@ void Join::joinBlockImpl( * For ASOF, the last column is used as the ASOF column */ ColumnsWithTypeAndName extras; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + if constexpr (is_asof_join) extras.push_back(right_table_keys.getByName(key_names_right.back())); AddedColumns added_columns(sample_block_with_columns_to_add, block_with_columns_to_add, block, saved_block_sample, extras, *this, key_columns, key_sizes, null_map); + if constexpr (need_replication) + added_columns.offsets_to_replicate = std::make_unique(block.rows()); + switchJoinRightColumns(maps_, added_columns, type); IColumn::Filter & row_filter = added_columns.filter; for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); - /// Filter & insert missing rows - constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; - constexpr bool inner_or_right = static_in_v; - std::vector right_keys_to_replicate [[maybe_unused]]; - if constexpr (!is_all_join && inner_or_right) + if constexpr (need_filter) { /// If ANY INNER | RIGHT JOIN - filter all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) @@ -883,16 +929,14 @@ void Join::joinBlockImpl( ColumnPtr thin_column = filterWithBlanks(col.column, filter); block.insert(correctNullability({thin_column, col.type, right_key.name}, is_nullable, null_map_filter)); - if constexpr (is_all_join) + if constexpr (need_replication) right_keys_to_replicate.push_back(block.getPositionByName(right_key.name)); } } - if constexpr (is_all_join) + if constexpr (need_replication) { std::unique_ptr & offsets_to_replicate = added_columns.offsets_to_replicate; - if (!offsets_to_replicate) - throw Exception("No data to filter columns", ErrorCodes::LOGICAL_ERROR); /// If ALL ... JOIN - we replicate all the columns except the new ones. for (size_t i = 0; i < existing_columns; ++i) @@ -982,7 +1026,7 @@ DataTypePtr Join::joinGetReturnType(const String & column_name) const template void Join::joinGetImpl(Block & block, const String & column_name, const Maps & maps_) const { - joinBlockImpl( + joinBlockImpl( block, {block.getByPosition(0).name}, {sample_block_with_columns_to_add.getByName(column_name)}, maps_); } @@ -999,9 +1043,10 @@ void Join::joinGet(Block & block, const String & column_name) const checkTypeOfKey(block, right_table_keys); - if (kind == ASTTableJoin::Kind::Left && strictness == ASTTableJoin::Strictness::Any) + if ((strictness == ASTTableJoin::Strictness::Any || strictness == ASTTableJoin::Strictness::RightAny) && + kind == ASTTableJoin::Kind::Left) { - joinGetImpl(block, column_name, std::get(maps)); + joinGetImpl(block, column_name, std::get(maps)); } else throw Exception("joinGet only supports StorageJoin of type Left Any", ErrorCodes::LOGICAL_ERROR); @@ -1039,7 +1084,7 @@ template struct AdderNonJoined; template -struct AdderNonJoined +struct AdderNonJoined { static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right) { @@ -1053,6 +1098,17 @@ struct AdderNonJoined } }; +template +struct AdderNonJoined +{ + static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right) + { + constexpr bool mapped_one = std::is_same_v || std::is_same_v; + if constexpr (!mapped_one) + AdderNonJoined::add(mapped, rows_added, columns_right); + } +}; + template struct AdderNonJoined { diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 3553680b879..58ae04f25d8 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -45,6 +45,16 @@ struct WithFlags : T mutable std::atomic used {}; void setUsed() const { used.store(true, std::memory_order_relaxed); } /// Could be set simultaneously from different threads. bool getUsed() const { return used; } + + bool setUsedOnce() const + { + /// fast check to prevent heavy CAS with seq_cst order + if (used.load(std::memory_order_relaxed)) + return false; + + bool expected = false; + return used.compare_exchange_strong(expected, true); + } }; template @@ -55,13 +65,14 @@ struct WithFlags : T void setUsed() const {} bool getUsed() const { return true; } + bool setUsedOnce() const { return true; } }; -using MappedAny = WithFlags; -using MappedAll = WithFlags; -using MappedAnyFull = WithFlags; -using MappedAllFull = WithFlags; -using MappedAsof = WithFlags; +using MappedOne = WithFlags; +using MappedAll = WithFlags; +using MappedOneFlagged = WithFlags; +using MappedAllFlagged = WithFlags; +using MappedAsof = WithFlags; } @@ -265,13 +276,13 @@ public: } }; - using MapsAny = MapsTemplate; + using MapsOne = MapsTemplate; using MapsAll = MapsTemplate; - using MapsAnyFull = MapsTemplate; - using MapsAllFull = MapsTemplate; + using MapsOneFlagged = MapsTemplate; + using MapsAllFlagged = MapsTemplate; using MapsAsof = MapsTemplate; - using MapsVariant = std::variant; + using MapsVariant = std::variant; private: friend class NonJoinedBlockInputStream; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 935ac67808a..0c9355c924e 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -540,7 +540,7 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const } } -void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin & out_table_join) +void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, bool old_any, ASTTableJoin & out_table_join) { const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node) @@ -560,6 +560,9 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul DB::ErrorCodes::EXPECTED_ALL_OR_ANY); } + if (old_any && table_join.strictness == ASTTableJoin::Strictness::Any) + table_join.strictness = ASTTableJoin::Strictness::RightAny; + out_table_join = table_join; } @@ -628,13 +631,8 @@ void checkJoin(const ASTTablesInSelectQueryElement * join) const auto & table_join = join->table_join->as(); if (table_join.strictness == ASTTableJoin::Strictness::Any) - if (table_join.kind != ASTTableJoin::Kind::Left) - throw Exception("Old ANY INNER|RIGHT|FULL JOINs are disabled by default. Their logic would be changed. " - "Old logic is many-to-one for all kinds of ANY JOINs. It's equil to apply distinct for right table keys. " - "Default bahaviour is reserved for many-to-one LEFT JOIN, one-to-many RIGHT JOIN and one-to-one INNER JOIN. " - "It would be equal to apply distinct for keys to right, left and both tables respectively. " - "Set any_join_distinct_right_table_keys=1 to enable old bahaviour.", - ErrorCodes::NOT_IMPLEMENTED); + if (table_join.kind == ASTTableJoin::Kind::Full) + throw Exception("ANY FULL JOINs are not implemented.", ErrorCodes::NOT_IMPLEMENTED); } std::vector getAggregates(const ASTPtr & query) @@ -958,7 +956,8 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Push the predicate expression down to the subqueries. result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); - setJoinStrictness(*select_query, settings.join_default_strictness, result.analyzed_join->table_join); + setJoinStrictness(*select_query, settings.join_default_strictness, settings.any_join_distinct_right_table_keys, + result.analyzed_join->table_join); collectJoinedColumns(*result.analyzed_join, *select_query, source_columns_set, result.aliases); } diff --git a/dbms/src/Interpreters/joinDispatch.h b/dbms/src/Interpreters/joinDispatch.h index 438641590d2..e5bb644f504 100644 --- a/dbms/src/Interpreters/joinDispatch.h +++ b/dbms/src/Interpreters/joinDispatch.h @@ -12,54 +12,33 @@ namespace DB { -template -struct MapGetterImpl; +template +struct MapGetter; -template <> -struct MapGetterImpl -{ - using Map = Join::MapsAny; -}; +template <> struct MapGetter { using Map = Join::MapsOne; }; +template <> struct MapGetter { using Map = Join::MapsOne; }; +template <> struct MapGetter { using Map = Join::MapsOneFlagged; }; +template <> struct MapGetter { using Map = Join::MapsOneFlagged; }; -template <> -struct MapGetterImpl -{ - using Map = Join::MapsAnyFull; -}; +template <> struct MapGetter { using Map = Join::MapsOne; }; +template <> struct MapGetter { using Map = Join::MapsOneFlagged; }; +template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; +template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; -template <> -struct MapGetterImpl -{ - using Map = Join::MapsAll; -}; +template <> struct MapGetter { using Map = Join::MapsAll; }; +template <> struct MapGetter { using Map = Join::MapsAll; }; +template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; +template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; -template <> -struct MapGetterImpl -{ - using Map = Join::MapsAllFull; -}; - -template -struct MapGetterImpl +template +struct MapGetter { using Map = Join::MapsAsof; }; -template -struct KindTrait -{ - // Affects the Adder trait so that when the right part is empty, adding a default value on the left - static constexpr bool fill_left = static_in_v; - // Affects the Map trait so that a `used` flag is attached to map slots in order to - // generate default values on the right when the left part is empty - static constexpr bool fill_right = static_in_v; -}; - -template -using Map = typename MapGetterImpl::fill_right, strictness>::Map; - -static constexpr std::array STRICTNESSES = { +static constexpr std::array STRICTNESSES = { + ASTTableJoin::Strictness::RightAny, ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All, ASTTableJoin::Strictness::Asof @@ -81,7 +60,7 @@ inline bool joinDispatchInit(ASTTableJoin::Kind kind, ASTTableJoin::Strictness s constexpr auto j = ij % STRICTNESSES.size(); if (kind == KINDS[i] && strictness == STRICTNESSES[j]) { - maps = Map(); + maps = typename MapGetter::Map(); return true; } return false; @@ -103,7 +82,7 @@ inline bool joinDispatch(ASTTableJoin::Kind kind, ASTTableJoin::Strictness stric func( std::integral_constant(), std::integral_constant(), - std::get>(maps)); + std::get::Map>(maps)); return true; } return false; diff --git a/dbms/src/Parsers/ASTTablesInSelectQuery.cpp b/dbms/src/Parsers/ASTTablesInSelectQuery.cpp index 47be2008284..9fc130adb31 100644 --- a/dbms/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/dbms/src/Parsers/ASTTablesInSelectQuery.cpp @@ -140,6 +140,7 @@ void ASTTableJoin::formatImplBeforeTable(const FormatSettings & settings, Format { case Strictness::Unspecified: break; + case Strictness::RightAny: case Strictness::Any: settings.ostr << "ANY "; break; diff --git a/dbms/src/Parsers/ASTTablesInSelectQuery.h b/dbms/src/Parsers/ASTTablesInSelectQuery.h index 9691dee96fa..713c3964ad2 100644 --- a/dbms/src/Parsers/ASTTablesInSelectQuery.h +++ b/dbms/src/Parsers/ASTTablesInSelectQuery.h @@ -74,7 +74,8 @@ struct ASTTableJoin : public IAST enum class Strictness { Unspecified, - Any, /// If there are many suitable rows to join, use any from them (also known as unique JOIN). + RightAny, /// Right ANY. If there are many suitable rows in right table, use any from them to join. + Any, /// Semi Join with any value from filtering table. For LEFT JOIN with Any and RightAny are the same. All, /// If there are many suitable rows to join, use all of them and replicate rows of "left" table (usual semantic of JOIN). Asof, /// For the last JOIN column, pick the latest value }; diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index d3f41f3b0b5..6ab7da7ab96 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -99,7 +99,7 @@ void registerStorageJoin(StorageFactory & factory) const String strictness_str = Poco::toLower(*opt_strictness_id); ASTTableJoin::Strictness strictness; if (strictness_str == "any") - strictness = ASTTableJoin::Strictness::Any; + strictness = ASTTableJoin::Strictness::RightAny; else if (strictness_str == "all") strictness = ASTTableJoin::Strictness::All; else @@ -329,7 +329,7 @@ private: for (; it != end; ++it) { - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Any) + if constexpr (STRICTNESS == ASTTableJoin::Strictness::RightAny) { for (size_t j = 0; j < columns.size(); ++j) if (j == key_pos) @@ -338,6 +338,11 @@ private: columns[j]->insertFrom(*it->getSecond().block->getByPosition(column_indices[j]).column.get(), it->getSecond().row_num); ++rows_added; } + else if constexpr (STRICTNESS == ASTTableJoin::Strictness::Any) + { + throw Exception("New ANY join storage is not implemented yet (set any_join_distinct_right_table_keys=1 to use old one)", + ErrorCodes::NOT_IMPLEMENTED); + } else if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) { throw Exception("ASOF join storage is not implemented yet", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/tests/queries/0_stateless/01031_new_any_join.reference b/dbms/tests/queries/0_stateless/01031_new_any_join.reference new file mode 100644 index 00000000000..f2e2119503d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01031_new_any_join.reference @@ -0,0 +1,32 @@ +any left +0 a1 0 +1 a2 0 +2 a3 2 b1 +3 a4 0 +4 a5 4 b3 +any left (rev) +2 a3 2 b1 +2 a3 2 b2 +4 a5 4 b3 +4 a5 4 b4 +4 a5 4 b5 +4 a5 4 b6 +any inner +2 a3 2 b1 +4 a5 4 b3 +any inner (rev) +2 a3 2 b1 +4 a5 4 b3 +any right +2 a3 2 b1 +2 a3 2 b2 +4 a5 4 b3 +4 a5 4 b4 +4 a5 4 b5 +4 a5 4 b6 +any right (rev) +0 a1 0 +1 a2 0 +2 a3 2 b1 +3 a4 0 +4 a5 4 b3 diff --git a/dbms/tests/queries/0_stateless/01031_new_any_join.sql b/dbms/tests/queries/0_stateless/01031_new_any_join.sql new file mode 100644 index 00000000000..822895c520f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01031_new_any_join.sql @@ -0,0 +1,32 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (x UInt32, s String) engine = Memory; +CREATE TABLE t2 (x UInt32, s String) engine = Memory; + +INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'); +INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (4, 'b6'); + +SET join_use_nulls = 0; +SET any_join_distinct_right_table_keys = 0; + +SELECT 'any left'; +SELECT t1.*, t2.* FROM t1 ANY LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any left (rev)'; +SELECT t1.*, t2.* FROM t2 ANY LEFT JOIN t1 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any inner'; +SELECT t1.*, t2.* FROM t1 ANY INNER JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any inner (rev)'; +SELECT t1.*, t2.* FROM t2 ANY INNER JOIN t1 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any right'; +SELECT t1.*, t2.* FROM t1 ANY RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'any right (rev)'; +SELECT t1.*, t2.* FROM t2 ANY RIGHT JOIN t1 USING(x) ORDER BY t1.x, t2.x; + +DROP TABLE t1; +DROP TABLE t2; From ac1b6f71cafd30f635831c15e44037be43064ed8 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 8 Nov 2019 13:49:14 +0300 Subject: [PATCH 05/77] update tests --- dbms/tests/queries/0_stateless/00203_full_join.reference | 2 ++ dbms/tests/queries/0_stateless/00203_full_join.sql | 2 +- .../00819_full_join_wrong_columns_in_block.reference | 3 +++ .../0_stateless/00819_full_join_wrong_columns_in_block.sql | 4 +++- 4 files changed, 9 insertions(+), 2 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00203_full_join.reference b/dbms/tests/queries/0_stateless/00203_full_join.reference index eedd5818063..d97597d17d3 100644 --- a/dbms/tests/queries/0_stateless/00203_full_join.reference +++ b/dbms/tests/queries/0_stateless/00203_full_join.reference @@ -43,3 +43,5 @@ Hello [0,1,2] 5 6 7 ddd 2 3 4 bbb ccc 5 6 7 ddd +2 3 4 bbb ccc +5 6 7 ddd diff --git a/dbms/tests/queries/0_stateless/00203_full_join.sql b/dbms/tests/queries/0_stateless/00203_full_join.sql index 0250aef35eb..9b07e9c84e7 100644 --- a/dbms/tests/queries/0_stateless/00203_full_join.sql +++ b/dbms/tests/queries/0_stateless/00203_full_join.sql @@ -27,7 +27,7 @@ SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k SET any_join_distinct_right_table_keys = 0; SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY FULL JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -- { serverError 48 } -SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; -- { serverError 48 } +SELECT k1, k2, k3, val_t1, val_t2 FROM t1_00203 ANY RIGHT JOIN t2_00203 USING (k3, k1, k2 AS k2_alias) ORDER BY k1, k2, k3; DROP TABLE t1_00203; DROP TABLE t2_00203; diff --git a/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference index 074ee47e294..afbe9855519 100644 --- a/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference +++ b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.reference @@ -6,3 +6,6 @@ 1 x x 1 x x 1 x x +1 x x +1 x x +1 x x diff --git a/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql index cf1c0bfe1f7..14a7424e634 100644 --- a/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql +++ b/dbms/tests/queries/0_stateless/00819_full_join_wrong_columns_in_block.sql @@ -12,5 +12,7 @@ SELECT * FROM (SELECT 1 AS a, 'x' AS b) any full join (SELECT 1 as a, 'y' as b) SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a; SET any_join_distinct_right_table_keys = 0; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) any join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) left join (SELECT 1 as a, 'y' as b) using a; +SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a; SELECT * FROM (SELECT 1 AS a, 'x' AS b) any full join (SELECT 1 as a, 'y' as b) using a; -- { serverError 48 } -SELECT * FROM (SELECT 1 AS a, 'x' AS b) any right join (SELECT 1 as a, 'y' as b) using a; -- { serverError 48 } From 3a24916ec2f8aed8922d8bd87d86d003a2a8aea1 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 8 Nov 2019 14:15:20 +0300 Subject: [PATCH 06/77] insertManyDefaults() overrides --- dbms/src/Columns/ColumnDecimal.h | 1 + dbms/src/Columns/ColumnFixedString.h | 5 +++++ dbms/src/Columns/ColumnString.h | 7 +++++++ dbms/src/Columns/ColumnVector.h | 5 +++++ 4 files changed, 18 insertions(+) diff --git a/dbms/src/Columns/ColumnDecimal.h b/dbms/src/Columns/ColumnDecimal.h index ad9d00661a0..86015371137 100644 --- a/dbms/src/Columns/ColumnDecimal.h +++ b/dbms/src/Columns/ColumnDecimal.h @@ -96,6 +96,7 @@ public: void insertFrom(const IColumn & src, size_t n) override { data.push_back(static_cast(src).getData()[n]); } void insertData(const char * pos, size_t /*length*/) override; void insertDefault() override { data.push_back(T()); } + virtual void insertManyDefaults(size_t length) override { data.resize_fill(data.size() + length); } void insert(const Field & x) override { data.push_back(DB::get>(x)); } void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index 91f0e92c0a9..a91a82d8524 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -92,6 +92,11 @@ public: chars.resize_fill(chars.size() + n); } + virtual void insertManyDefaults(size_t length) override + { + chars.resize_fill(chars.size() + n * length); + } + void popBack(size_t elems) override { chars.resize_assume_reserved(chars.size() - n * elems); diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 7c686f79767..8f1eced92f9 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -205,6 +205,13 @@ public: offsets.push_back(offsets.back() + 1); } + virtual void insertManyDefaults(size_t length) override + { + chars.resize_fill(chars.size() + length); + for (size_t i = 0; i < length; ++i) + offsets.push_back(offsets.back() + 1); + } + int compareAt(size_t n, size_t m, const IColumn & rhs_, int /*nan_direction_hint*/) const override { const ColumnString & rhs = assert_cast(rhs_); diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 28307cb33f0..f80ec6f0056 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -144,6 +144,11 @@ public: data.push_back(T()); } + virtual void insertManyDefaults(size_t length) override + { + data.resize_fill(data.size() + length, T()); + } + void popBack(size_t n) override { data.resize_assume_reserved(data.size() - n); From 6f40bcfcee4f21a35aa6e294bce46fedc31200f8 Mon Sep 17 00:00:00 2001 From: chertus Date: Fri, 8 Nov 2019 19:13:43 +0300 Subject: [PATCH 07/77] perf optimisation --- dbms/src/Interpreters/Join.cpp | 31 +++++++++++++++++-------------- 1 file changed, 17 insertions(+), 14 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 4dffee8aa41..33091c2f218 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -632,9 +632,12 @@ public: return ColumnWithTypeAndName(std::move(columns[i]), type_name[i].first, type_name[i].second); } + template void appendFromBlock(const Block & block, size_t row_num) { - applyLazyDefaults(); + if constexpr (has_defaults) + applyLazyDefaults(); + for (size_t j = 0; j < right_indexes.size(); ++j) columns[j]->insertFrom(*block.getByPosition(right_indexes[j]).column, row_num); } @@ -676,20 +679,20 @@ private: } }; -template +template void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset) { for (auto it = mapped.begin(); it.ok(); ++it) { - added.appendFromBlock(*it->block, it->row_num); + added.appendFromBlock(*it->block, it->row_num); ++current_offset; } }; -template +template void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & current_offset [[maybe_unused]]) { - if constexpr (_add_missing) + if constexpr (add_missing) { added.appendDefaultRow(); ++current_offset; @@ -708,7 +711,7 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) constexpr bool left_or_full = static_in_v; constexpr bool right = KIND == ASTTableJoin::Kind::Right; - constexpr bool _add_missing = left_or_full; + constexpr bool add_missing = left_or_full; constexpr bool need_replication = is_all_join || (is_any_join && right); size_t rows = added_columns.rows_to_add; @@ -728,7 +731,7 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) { if (_has_null_map && (*added_columns.null_map)[i]) { - addNotFoundRow<_add_missing>(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); } else { @@ -745,19 +748,19 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) { filter[i] = 1; mapped.setUsed(); - added_columns.appendFromBlock(*found->block, found->row_num); + added_columns.appendFromBlock(*found->block, found->row_num); } else { filter[i] = 0; - addNotFoundRow<_add_missing>(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); } } else if constexpr (is_all_join) { filter[i] = 1; mapped.setUsed(); - addFoundRowAll(mapped, added_columns, current_offset); + addFoundRowAll(mapped, added_columns, current_offset); } else if constexpr (is_any_join && right) { @@ -765,7 +768,7 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) if (mapped.setUsedOnce()) { filter[i] = 1; - addFoundRowAll(mapped, added_columns, current_offset); + addFoundRowAll(mapped, added_columns, current_offset); } } else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Inner) @@ -774,7 +777,7 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) if (mapped.setUsedOnce()) { filter[i] = 1; - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); } } else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Full) @@ -785,11 +788,11 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) { filter[i] = 1; mapped.setUsed(); - added_columns.appendFromBlock(*mapped.block, mapped.row_num); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); } } else - addNotFoundRow<_add_missing>(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); } if constexpr (need_replication) From b51e6f9fe37e12877dd12313c9ed2604da63af97 Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 11 Nov 2019 14:42:10 +0300 Subject: [PATCH 08/77] trying to remove perf degradation --- dbms/src/Interpreters/Join.cpp | 145 ++++++++++++++++----------------- 1 file changed, 72 insertions(+), 73 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 33091c2f218..00f53d38859 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -594,14 +594,11 @@ public: const ColumnsWithTypeAndName & extras, const Join & join_, const ColumnRawPtrs & key_columns_, - const Sizes & key_sizes_, - ConstNullMapPtr null_map_) + const Sizes & key_sizes_) : join(join_) , key_columns(key_columns_) , key_sizes(key_sizes_) - , null_map(null_map_) , rows_to_add(block.rows()) - , filter(rows_to_add, 0) { size_t num_columns_to_add = sample_block_with_columns_to_add.columns(); @@ -660,9 +657,7 @@ public: const Join & join; const ColumnRawPtrs & key_columns; const Sizes & key_sizes; - ConstNullMapPtr null_map; size_t rows_to_add; - IColumn::Filter filter; std::unique_ptr offsets_to_replicate; private: @@ -703,7 +698,7 @@ void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & cur /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). template -void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) +NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added_columns, const ConstNullMapPtr & null_map [[maybe_unused]]) { constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; @@ -715,10 +710,12 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) constexpr bool need_replication = is_all_join || (is_any_join && right); size_t rows = added_columns.rows_to_add; - - IColumn::Filter & filter = added_columns.filter; + IColumn::Filter filter(rows, 0); Arena pool; + if constexpr (need_replication) + added_columns.offsets_to_replicate = std::make_unique(rows); + const IColumn * asof_column [[maybe_unused]] = nullptr; if constexpr (is_asof_join) asof_column = extractAsofColumn(added_columns.key_columns); @@ -729,98 +726,104 @@ void NO_INLINE joinRightColumns(const Map & map, AddedColumns & added_columns) for (size_t i = 0; i < rows; ++i) { - if (_has_null_map && (*added_columns.null_map)[i]) + if constexpr (_has_null_map) { - addNotFoundRow(added_columns, current_offset); - } - else - { - auto find_result = key_getter.findKey(map, i, pool); - - if (find_result.isFound()) + if ((*null_map)[i]) { - auto & mapped = find_result.getMapped(); + addNotFoundRow(added_columns, current_offset); - if constexpr (is_asof_join) - { - const Join & join = added_columns.join; - if (const RowRef * found = mapped.findAsof(join.getAsofType(), join.getAsofInequality(), asof_column, i)) - { - filter[i] = 1; - mapped.setUsed(); - added_columns.appendFromBlock(*found->block, found->row_num); - } - else - { - filter[i] = 0; - addNotFoundRow(added_columns, current_offset); - } - } - else if constexpr (is_all_join) + if constexpr (need_replication) + (*added_columns.offsets_to_replicate)[i] = current_offset; + continue; + } + } + + auto find_result = key_getter.findKey(map, i, pool); + + if (find_result.isFound()) + { + auto & mapped = find_result.getMapped(); + + if constexpr (is_asof_join) + { + const Join & join = added_columns.join; + if (const RowRef * found = mapped.findAsof(join.getAsofType(), join.getAsofInequality(), asof_column, i)) { filter[i] = 1; mapped.setUsed(); + added_columns.appendFromBlock(*found->block, found->row_num); + } + else + { + filter[i] = 0; + addNotFoundRow(added_columns, current_offset); + } + } + else if constexpr (is_all_join) + { + filter[i] = 1; + mapped.setUsed(); + addFoundRowAll(mapped, added_columns, current_offset); + } + else if constexpr (is_any_join && right) + { + /// Use first appered left key + it needs left columns replication + if (mapped.setUsedOnce()) + { + filter[i] = 1; addFoundRowAll(mapped, added_columns, current_offset); } - else if constexpr (is_any_join && right) - { - /// Use first appered left key + it needs left columns replication - if (mapped.setUsedOnce()) - { - filter[i] = 1; - addFoundRowAll(mapped, added_columns, current_offset); - } - } - else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Inner) - { - /// Use first appered left key only - if (mapped.setUsedOnce()) - { - filter[i] = 1; - added_columns.appendFromBlock(*mapped.block, mapped.row_num); - } - } - else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Full) - { - /// TODO - } - else /// ANY LEFT + old ANY (RightAny) + } + else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Inner) + { + /// Use first appered left key only + if (mapped.setUsedOnce()) { filter[i] = 1; - mapped.setUsed(); added_columns.appendFromBlock(*mapped.block, mapped.row_num); } } - else - addNotFoundRow(added_columns, current_offset); + else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Full) + { + /// TODO + } + else /// ANY LEFT + old ANY (RightAny) + { + filter[i] = 1; + mapped.setUsed(); + added_columns.appendFromBlock(*mapped.block, mapped.row_num); + } } + else + addNotFoundRow(added_columns, current_offset); if constexpr (need_replication) (*added_columns.offsets_to_replicate)[i] = current_offset; } added_columns.applyLazyDefaults(); + return filter; } template -void joinRightColumnsSwitchNullability(const Map & map, AddedColumns & added_columns) +IColumn::Filter joinRightColumnsSwitchNullability(const Map & map, AddedColumns & added_columns, const ConstNullMapPtr & null_map) { - if (added_columns.null_map) - joinRightColumns(map, added_columns); + if (null_map) + return joinRightColumns(map, added_columns, null_map); else - joinRightColumns(map, added_columns); + return joinRightColumns(map, added_columns, nullptr); } template -void switchJoinRightColumns(const Maps & maps_, AddedColumns & added_columns, Join::Type type) +IColumn::Filter switchJoinRightColumns(const Maps & maps_, AddedColumns & added_columns, Join::Type type, const ConstNullMapPtr & null_map) { switch (type) { #define M(TYPE) \ case Join::Type::TYPE: \ - joinRightColumnsSwitchNullability>::Type>(\ - *maps_.TYPE, added_columns);\ + *maps_.TYPE, added_columns, null_map);\ break; APPLY_FOR_JOIN_VARIANTS(M) #undef M @@ -882,13 +885,9 @@ void Join::joinBlockImpl( extras.push_back(right_table_keys.getByName(key_names_right.back())); AddedColumns added_columns(sample_block_with_columns_to_add, block_with_columns_to_add, block, saved_block_sample, - extras, *this, key_columns, key_sizes, null_map); + extras, *this, key_columns, key_sizes); - if constexpr (need_replication) - added_columns.offsets_to_replicate = std::make_unique(block.rows()); - - switchJoinRightColumns(maps_, added_columns, type); - IColumn::Filter & row_filter = added_columns.filter; + IColumn::Filter row_filter = switchJoinRightColumns(maps_, added_columns, type, null_map); for (size_t i = 0; i < added_columns.size(); ++i) block.insert(added_columns.moveColumn(i)); From b380f8d53cba957f7be02a251edf971e001f266a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Fri, 15 Nov 2019 14:30:19 +0300 Subject: [PATCH 09/77] bug test added --- ...01030_incorrect_count_merge_tree.reference | 10 ++++++++ .../01030_incorrect_count_merge_tree.sql | 25 +++++++++++++++++++ 2 files changed, 35 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference create mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference new file mode 100644 index 00000000000..75378377541 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference @@ -0,0 +1,10 @@ +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql new file mode 100644 index 00000000000..ed69490fd8b --- /dev/null +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql @@ -0,0 +1,25 @@ +drop table if exists tst; + +create table tst (timestamp DateTime, val Nullable(Int8)) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); + +insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); + +select count() from tst; +select count() from tst final; + +select count() from tst where timestamp is not null; +select count() from tst final where timestamp is not null; + +select count() from tst where val is not null; +select count() from tst final where val is not null; + +select count() from tst final where timestamp>0; +select count() from tst final prewhere timestamp > 0; + +select count() from tst final where val>0; +select count() from tst final prewhere val>0; + +drop table tst; + + + From 8eebcad52deb395631a1057e6731927979ca0cf2 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 18 Nov 2019 13:14:37 +0300 Subject: [PATCH 10/77] experiments --- dbms/src/Common/PODArray.h | 6 ++++++ .../SummingSortedBlockInputStream.cpp | 20 +++++++++++++++++++ 2 files changed, 26 insertions(+) diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index def8f675c25..3dadca8a325 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -377,6 +377,12 @@ public: if (unlikely(this->c_end == this->c_end_of_storage)) this->reserveForNextSize(std::forward(allocator_params)...); + if (this->c_end == nullptr) + std::cout << "this->c_end == nullptr" << std::endl; + + if (this->c_end_of_storage == nullptr) + std::cout << "this->c_end_of_storage == nullptr" << std::endl; + new (t_end()) T(std::forward(x)); this->c_end += this->byte_size(1); } diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index 35fdc948239..a81eeef1a8b 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -48,6 +48,8 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( size_t max_block_size_) : MergingSortedBlockInputStream(inputs_, description_, max_block_size_) { + std::cout << "Constructor SummingSortedBlockInputStream()" << std::endl; + std::cout << "num_columns " << num_columns << std::endl; current_row.resize(num_columns); /// name of nested structure -> the column numbers that refer to it. @@ -60,11 +62,14 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( */ for (size_t i = 0; i < num_columns; ++i) { + std::cout << "Constructor for loop" << std::endl; const ColumnWithTypeAndName & column = header.safeGetByPosition(i); + std::cout << "column name " << column.name << std::endl; /// Discover nested Maps and find columns for summation if (typeid_cast(column.type.get())) { + std::cout << "typeid_cast(column.type.get())" << std::endl; const auto map_name = Nested::extractTableName(column.name); /// if nested table name ends with `Map` it is a possible candidate for special handling if (map_name == column.name || !endsWith(map_name, "Map")) @@ -81,6 +86,7 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( if (!column.type->isSummable() && !is_agg_func) { column_numbers_not_to_aggregate.push_back(i); + std::cout << "!column.type->isSummable() && !is_agg_func" << std::endl; continue; } @@ -106,11 +112,13 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( } columns_to_aggregate.emplace_back(std::move(desc)); + std::cout << "columns_to_aggregate" << std::endl; } else { // Column is not going to be summed, use last value column_numbers_not_to_aggregate.push_back(i); + std::cout << "column_numbers_not_to_aggregate" << std::endl; } } } @@ -258,6 +266,7 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me Block SummingSortedBlockInputStream::readImpl() { + std::cout << "SummingSortedBlockInputStream::readImpl" << std::endl; if (finished) return Block(); @@ -268,14 +277,20 @@ Block SummingSortedBlockInputStream::readImpl() throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); if (merged_columns.empty()) + { + std::cout << "merged_columns.empty()" << std::endl; return {}; + } + /// Update aggregation result columns for current block for (auto & desc : columns_to_aggregate) { + std::cout << "readImpl() in for loop" << std::endl; // Wrap aggregated columns in a tuple to match function signature if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) { + std::cout << "!desc.is_agg_func_type && isTuple(desc.function->getReturnType())" << std::endl; size_t tuple_size = desc.column_numbers.size(); MutableColumns tuple_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) @@ -284,7 +299,11 @@ Block SummingSortedBlockInputStream::readImpl() desc.merged_column = ColumnTuple::create(std::move(tuple_columns)); } else + { + std::cout << "else" << std::endl; desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty(); + } + } merge(merged_columns, queue_without_collation); @@ -310,6 +329,7 @@ Block SummingSortedBlockInputStream::readImpl() void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::priority_queue & queue) { + std::cout << "SummingSortedBlockInputStream::merge(...)" << std::endl; merged_rows = 0; /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` From 37b84977473bc60b9186bb2871b7bbe2cc791420 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 19 Nov 2019 17:28:57 +0300 Subject: [PATCH 11/77] add more tests --- ...01030_incorrect_count_merge_tree.reference | 15 +++------ .../01030_incorrect_count_merge_tree.sql | 33 +++++++------------ 2 files changed, 17 insertions(+), 31 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference index 75378377541..a5f040c71aa 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference @@ -1,10 +1,5 @@ -2 -2 -2 -2 -2 -2 -2 -2 -2 -2 +ClickHouse client version 19.17.1.1. +Connecting to localhost:9000 as user default. +Connected to ClickHouse server version 19.17.1 revision 54428. + +jakalletti :) \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql index ed69490fd8b..95e40d4c377 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql @@ -1,25 +1,16 @@ -drop table if exists tst; -create table tst (timestamp DateTime, val Nullable(Int8)) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); - -insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); - -select count() from tst; -select count() from tst final; - -select count() from tst where timestamp is not null; -select count() from tst final where timestamp is not null; - -select count() from tst where val is not null; -select count() from tst final where val is not null; - -select count() from tst final where timestamp>0; -select count() from tst final prewhere timestamp > 0; - -select count() from tst final where val>0; -select count() from tst final prewhere val>0; - -drop table tst; +drop table if exists tst2; +create table tst2 +( + timestamp DateTime, + val Nullable(Int8) +) engine MergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); +insert into tst2 values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); +select * from tst2; +select count() from tst2 where val is not null; +select count() from tst2 where timestamp is not null; +select count(*) from tst2 where timestamp > '2017-01-01 00:00:00' +drop table tst2; From b9ad1fc79aeaf99a98c9ecad3d6cee722024f858 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 19 Nov 2019 17:29:52 +0300 Subject: [PATCH 12/77] better --- dbms/src/Common/PODArray.h | 6 --- .../MergingSortedBlockInputStream.cpp | 4 ++ .../SummingSortedBlockInputStream.cpp | 26 ++++++++++++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 16 +++++++- ...030_incorrect_count_summing_merge_tree.sql | 41 +++++++++++++++++++ ...umming_merge_tree_with_nullables.reference | 10 +++++ 6 files changed, 95 insertions(+), 8 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql create mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index 3dadca8a325..def8f675c25 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -377,12 +377,6 @@ public: if (unlikely(this->c_end == this->c_end_of_storage)) this->reserveForNextSize(std::forward(allocator_params)...); - if (this->c_end == nullptr) - std::cout << "this->c_end == nullptr" << std::endl; - - if (this->c_end_of_storage == nullptr) - std::cout << "this->c_end_of_storage == nullptr" << std::endl; - new (t_end()) T(std::forward(x)); this->c_end += this->byte_size(1); } diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp index 8c0707e09b0..d43e608fed1 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -23,7 +23,11 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream( , average_block_sizes(average_block_sizes_), source_blocks(inputs_.size()) , cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_) { + std::cout << "size1 " << children.size() << std::endl; children.insert(children.end(), inputs_.begin(), inputs_.end()); + std::cout << "size2 " << children.size() << std::endl; + std::cout << children.at(0)->getName() << std::endl; + children.at(0)->dumpTree(std::cout, 0); header = children.at(0)->getHeader(); num_columns = header.columns(); } diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index a81eeef1a8b..4ce47963edf 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -50,6 +50,8 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( { std::cout << "Constructor SummingSortedBlockInputStream()" << std::endl; std::cout << "num_columns " << num_columns << std::endl; + std::cout << "inputs size " << inputs_.size() << std::endl; + std::cout << StackTrace().toString() << std::endl; current_row.resize(num_columns); /// name of nested structure -> the column numbers that refer to it. @@ -65,6 +67,7 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( std::cout << "Constructor for loop" << std::endl; const ColumnWithTypeAndName & column = header.safeGetByPosition(i); std::cout << "column name " << column.name << std::endl; + std::cout << header.dumpNames() << std::endl; /// Discover nested Maps and find columns for summation if (typeid_cast(column.type.get())) @@ -206,8 +209,13 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns) { + std::cout << "SummingSortedBlockInputStream::insertCurrentRowIfNeeded" << std::endl; + if (columns_to_aggregate.empty()) + current_row_is_zero = false; + for (auto & desc : columns_to_aggregate) { + std::cout << "MergedColumnName " << desc.merged_column->getName() << std::endl; // Do not insert if the aggregation state hasn't been created if (desc.created) { @@ -250,6 +258,7 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me /// (at this moment we need rollback only cols from columns_to_aggregate) if (current_row_is_zero) { + std::cout << "current_row_is_zero" << std::endl; for (auto & desc : columns_to_aggregate) desc.merged_column->popBack(1); @@ -261,6 +270,7 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me /// Update per-block and per-group flags ++merged_rows; + std::cout << "insertCurrentRowIfNeeded merged_rows " << merged_rows << std::endl; } @@ -306,9 +316,12 @@ Block SummingSortedBlockInputStream::readImpl() } + std::cout << "queue_without_collation.size() " << queue_without_collation.size() << std::endl; merge(merged_columns, queue_without_collation); Block res = header.cloneWithColumns(std::move(merged_columns)); + std::cout << "result rows count " << res.rows() << std::endl; + /// Place aggregation results into block. for (auto & desc : columns_to_aggregate) { @@ -335,19 +348,30 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` while (!queue.empty()) { + std::cout << "while loop" << std::endl; + SortCursor current = queue.top(); setPrimaryKeyRef(next_key, current); + if (next_key.empty()) + std::cout << "next_key empty()" << std::endl; + bool key_differs; if (current_key.empty()) /// The first key encountered. { + std::cout << "current_key is empty" << std::endl; key_differs = true; current_row_is_zero = true; } else + { key_differs = next_key != current_key; + /// If current_key is not empty - thats why current_row is not zero. + current_row_is_zero = false; + } + if (key_differs) { @@ -378,6 +402,7 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: /// We have only columns_to_aggregate. The status of current row will be determined /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. current_row_is_zero = true; + std::cout << "maps_to_sum.empty() true" << std::endl; } else { @@ -415,6 +440,7 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: /// If it is zero, and without it the output stream will be empty, we will write it anyway. insertCurrentRowIfNeeded(merged_columns); finished = true; + std::cout << "merged rows " << merged_rows << std::endl; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 60ed25ed43c..fcb20edc863 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -577,6 +577,9 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( { /// Add columns needed to calculate the sorting expression and the sign. std::vector add_columns = data.sorting_key_expr->getRequiredColumns(); + std::cout << "std::vector add_columns" << std::endl; + for (auto column: add_columns) + std::cout << column << std::endl; column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); if (!data.merging_params.sign_column.empty()) @@ -1112,13 +1115,22 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( BlockInputStreams streams; streams.reserve(num_streams); - for (size_t i = 0; i < num_streams; ++i) + for (size_t i = 0; i < num_streams; ++i) { + std::cout << "labmda for loop № " << i << std::endl; + std::cout << pipes[i].getHeader().dumpStructure() << std::endl; streams.emplace_back(std::make_shared(std::move(pipes[i]))); + } + pipes.clear(); return streams; }; + auto anime = streams_to_merge().at(0)->getHeader(); + std::cout << anime.dumpNames() << std::endl; + std::cout << anime.dumpStructure() << std::endl; + std::cout << anime.columns() << std::endl; + BlockInputStreamPtr merged; switch (data.merging_params.mode) { @@ -1137,7 +1149,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( case MergeTreeData::MergingParams::Summing: merged = std::make_shared(streams_to_merge(), - sort_description, data.merging_params.columns_to_sum, max_block_size); + sort_description, data.merging_params.columns_to_sum, max_block_size); break; case MergeTreeData::MergingParams::Aggregating: diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql new file mode 100644 index 00000000000..770e24f7d1f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql @@ -0,0 +1,41 @@ +drop table if exists tst; +create table tst (timestamp DateTime, val Nullable(Int8)) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); +insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); + +select count() from tst; +select count() from tst final; + +select count() from tst where timestamp is not null; +select count() from tst final where timestamp is not null; + +select count() from tst where val is not null; +select count() from tst final where val is not null; + +select count() from tst final where timestamp>0; +select count() from tst final prewhere timestamp > 0; +select count() from tst final where timestamp > '2017-01-01 00:00:00' + +select count() from tst final where val>0; +select count() from tst final prewhere val>0; + +drop table if exists tst; +create table tst (timestamp DateTime, val Int8) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); +insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); + +select count() from tst; +select count() from tst final; + +select count() from tst where timestamp is not null; +select count() from tst final where timestamp is not null; + +select count() from tst where val is not null; +select count() from tst final where val is not null; + +select count() from tst final where timestamp>0; +select count() from tst final prewhere timestamp > 0; +select count() from tst final where timestamp > '2017-01-01 00:00:00' + +select count() from tst final where val>0; +select count() from tst final prewhere val>0; + +drop table tst; diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference new file mode 100644 index 00000000000..75378377541 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference @@ -0,0 +1,10 @@ +2 +2 +2 +2 +2 +2 +2 +2 +2 +2 From 6d60d10461dc9f9d848df4930754d2d81f508d8e Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 19 Nov 2019 21:22:32 +0300 Subject: [PATCH 13/77] just to restart pending perf test --- dbms/src/Parsers/ASTTablesInSelectQuery.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Parsers/ASTTablesInSelectQuery.h b/dbms/src/Parsers/ASTTablesInSelectQuery.h index 713c3964ad2..57ba6ca93f2 100644 --- a/dbms/src/Parsers/ASTTablesInSelectQuery.h +++ b/dbms/src/Parsers/ASTTablesInSelectQuery.h @@ -166,5 +166,4 @@ struct ASTTablesInSelectQuery : public IAST void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override; }; - } From 194c25a2a764f1895da1c43d97b2b5685e049f6f Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 20 Nov 2019 12:15:52 +0300 Subject: [PATCH 14/77] CLICKHOUSEDOCS-466: EN docs for query_profiler* settings. --- contrib/poco | 2 +- docs/en/operations/settings/settings.md | 41 +++++++++++++++++++++++++ 2 files changed, 42 insertions(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 2b273bfe9db..6216cc01a10 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 2b273bfe9db89429b2040c024484dee0197e48c7 +Subproject commit 6216cc01a107ce149863411ca29013a224f80343 diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 5def2bee8c2..f44884c568d 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -994,4 +994,45 @@ Lower values mean higher priority. Threads with low `nice` priority values are e Default value: 0. + +## query_profiler_real_time_period_ns {#query_profiler_real_time_period_ns} + +Sets the period for a real clock timer of the query profiler. Real clock timer counts wall-clock time. + +Possible values: + +- Positive integer number of nanoseconds. + + Recommended values: + + - 10000000 (100 times a second) nanosecods and more for for single queries. + - 1000000000 (once a second) for cluster-wide profiling. + +- 0 for turning off the timer. + +Type: [UInt64](../../data_types/int_uint.md). + +Default value: 1000000000 nanoseconds. + + +## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} + +Sets the period for a CPU clock timer of the query profiler. Real clock timer counts CPU time. + +Possible values: + +- Positive integer number of nanoseconds. + + Recommended values: + + - 10000000 (100 times a second) nanosecods and more for for single queries. + - 1000000000 (once a second) for cluster-wide profiling. + +- 0 for turning off the timer. + +Type: [UInt64](../../data_types/int_uint.md). + +Default value: 1000000000 nanoseconds. + + [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) From 9c6e024e4cc4c52c111c3951542774fc70dcd550 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Wed, 20 Nov 2019 13:05:50 +0300 Subject: [PATCH 15/77] CLICKHOUSEDOCS-466: Links --- docs/en/operations/settings/settings.md | 7 +++++++ docs/en/operations/system_tables.md | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index f44884c568d..755ca30793c 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1014,6 +1014,9 @@ Type: [UInt64](../../data_types/int_uint.md). Default value: 1000000000 nanoseconds. +**See Also** + +- [system.trace_log](../system_tables.md#system_tables-trace_log) ## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} @@ -1034,5 +1037,9 @@ Type: [UInt64](../../data_types/int_uint.md). Default value: 1000000000 nanoseconds. +**See Also** + +- [system.trace_log](../system_tables.md#system_tables-trace_log) + [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index e85d5225763..eb48327fe3b 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -490,7 +490,7 @@ You can specify an arbitrary partitioning key for the `system.query_log` table i Contains stack traces collected by the sampling query profiler. -ClickHouse creates this table when the [trace_log](server_settings/settings.md#server_settings-trace_log) server configuration section is set. Also the `query_profiler_real_time_period_ns` and `query_profiler_cpu_time_period_ns` settings should be set. +ClickHouse creates this table when the [trace_log](server_settings/settings.md#server_settings-trace_log) server configuration section is set. Also the [query_profiler_real_time_period_ns](settings/settings.md#query_profiler_real_time_period_ns) and [query_profiler_cpu_time_period_ns](settings/settings.md#query_profiler_cpu_time_period_ns) settings should be set. To analyze logs, use the `addressToLine`, `addressToSymbol` and `demangle` introspection functions. From 8f96118c14412c25de24aa41cf735c8fcfffe89c Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 25 Nov 2019 14:14:27 +0300 Subject: [PATCH 16/77] useless --- dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index fcb20edc863..5048e510bbc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1126,11 +1126,6 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( return streams; }; - auto anime = streams_to_merge().at(0)->getHeader(); - std::cout << anime.dumpNames() << std::endl; - std::cout << anime.dumpStructure() << std::endl; - std::cout << anime.columns() << std::endl; - BlockInputStreamPtr merged; switch (data.merging_params.mode) { From ee0855be6d100f383562948dec9e15b4f309254d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Mon, 25 Nov 2019 20:42:17 +0300 Subject: [PATCH 17/77] remove couts --- .../MergingSortedBlockInputStream.cpp | 3 -- .../SummingSortedBlockInputStream.cpp | 36 ------------------- dbms/src/Processors/ISource.cpp | 1 - .../MergeTree/MergeTreeDataSelectExecutor.cpp | 9 +---- 4 files changed, 1 insertion(+), 48 deletions(-) diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp index d43e608fed1..0c9055287a2 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -23,10 +23,7 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream( , average_block_sizes(average_block_sizes_), source_blocks(inputs_.size()) , cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_) { - std::cout << "size1 " << children.size() << std::endl; children.insert(children.end(), inputs_.begin(), inputs_.end()); - std::cout << "size2 " << children.size() << std::endl; - std::cout << children.at(0)->getName() << std::endl; children.at(0)->dumpTree(std::cout, 0); header = children.at(0)->getHeader(); num_columns = header.columns(); diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index 4ce47963edf..224475af636 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -48,10 +48,6 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( size_t max_block_size_) : MergingSortedBlockInputStream(inputs_, description_, max_block_size_) { - std::cout << "Constructor SummingSortedBlockInputStream()" << std::endl; - std::cout << "num_columns " << num_columns << std::endl; - std::cout << "inputs size " << inputs_.size() << std::endl; - std::cout << StackTrace().toString() << std::endl; current_row.resize(num_columns); /// name of nested structure -> the column numbers that refer to it. @@ -64,15 +60,11 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( */ for (size_t i = 0; i < num_columns; ++i) { - std::cout << "Constructor for loop" << std::endl; const ColumnWithTypeAndName & column = header.safeGetByPosition(i); - std::cout << "column name " << column.name << std::endl; - std::cout << header.dumpNames() << std::endl; /// Discover nested Maps and find columns for summation if (typeid_cast(column.type.get())) { - std::cout << "typeid_cast(column.type.get())" << std::endl; const auto map_name = Nested::extractTableName(column.name); /// if nested table name ends with `Map` it is a possible candidate for special handling if (map_name == column.name || !endsWith(map_name, "Map")) @@ -89,7 +81,6 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( if (!column.type->isSummable() && !is_agg_func) { column_numbers_not_to_aggregate.push_back(i); - std::cout << "!column.type->isSummable() && !is_agg_func" << std::endl; continue; } @@ -115,13 +106,11 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( } columns_to_aggregate.emplace_back(std::move(desc)); - std::cout << "columns_to_aggregate" << std::endl; } else { // Column is not going to be summed, use last value column_numbers_not_to_aggregate.push_back(i); - std::cout << "column_numbers_not_to_aggregate" << std::endl; } } } @@ -209,13 +198,11 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns) { - std::cout << "SummingSortedBlockInputStream::insertCurrentRowIfNeeded" << std::endl; if (columns_to_aggregate.empty()) current_row_is_zero = false; for (auto & desc : columns_to_aggregate) { - std::cout << "MergedColumnName " << desc.merged_column->getName() << std::endl; // Do not insert if the aggregation state hasn't been created if (desc.created) { @@ -258,7 +245,6 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me /// (at this moment we need rollback only cols from columns_to_aggregate) if (current_row_is_zero) { - std::cout << "current_row_is_zero" << std::endl; for (auto & desc : columns_to_aggregate) desc.merged_column->popBack(1); @@ -270,13 +256,11 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me /// Update per-block and per-group flags ++merged_rows; - std::cout << "insertCurrentRowIfNeeded merged_rows " << merged_rows << std::endl; } Block SummingSortedBlockInputStream::readImpl() { - std::cout << "SummingSortedBlockInputStream::readImpl" << std::endl; if (finished) return Block(); @@ -287,20 +271,15 @@ Block SummingSortedBlockInputStream::readImpl() throw Exception("Logical error: " + getName() + " does not support collations", ErrorCodes::LOGICAL_ERROR); if (merged_columns.empty()) - { - std::cout << "merged_columns.empty()" << std::endl; return {}; - } /// Update aggregation result columns for current block for (auto & desc : columns_to_aggregate) { - std::cout << "readImpl() in for loop" << std::endl; // Wrap aggregated columns in a tuple to match function signature if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType())) { - std::cout << "!desc.is_agg_func_type && isTuple(desc.function->getReturnType())" << std::endl; size_t tuple_size = desc.column_numbers.size(); MutableColumns tuple_columns(tuple_size); for (size_t i = 0; i < tuple_size; ++i) @@ -309,19 +288,13 @@ Block SummingSortedBlockInputStream::readImpl() desc.merged_column = ColumnTuple::create(std::move(tuple_columns)); } else - { - std::cout << "else" << std::endl; desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty(); - } } - std::cout << "queue_without_collation.size() " << queue_without_collation.size() << std::endl; merge(merged_columns, queue_without_collation); Block res = header.cloneWithColumns(std::move(merged_columns)); - std::cout << "result rows count " << res.rows() << std::endl; - /// Place aggregation results into block. for (auto & desc : columns_to_aggregate) { @@ -342,26 +315,19 @@ Block SummingSortedBlockInputStream::readImpl() void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std::priority_queue & queue) { - std::cout << "SummingSortedBlockInputStream::merge(...)" << std::endl; merged_rows = 0; /// Take the rows in needed order and put them in `merged_columns` until rows no more than `max_block_size` while (!queue.empty()) { - std::cout << "while loop" << std::endl; - SortCursor current = queue.top(); setPrimaryKeyRef(next_key, current); - if (next_key.empty()) - std::cout << "next_key empty()" << std::endl; - bool key_differs; if (current_key.empty()) /// The first key encountered. { - std::cout << "current_key is empty" << std::endl; key_differs = true; current_row_is_zero = true; } @@ -402,7 +368,6 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: /// We have only columns_to_aggregate. The status of current row will be determined /// in 'insertCurrentRowIfNeeded' method on the values of aggregate functions. current_row_is_zero = true; - std::cout << "maps_to_sum.empty() true" << std::endl; } else { @@ -440,7 +405,6 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: /// If it is zero, and without it the output stream will be empty, we will write it anyway. insertCurrentRowIfNeeded(merged_columns); finished = true; - std::cout << "merged rows " << merged_rows << std::endl; } diff --git a/dbms/src/Processors/ISource.cpp b/dbms/src/Processors/ISource.cpp index d40f0e32fb7..dcda869e1d4 100644 --- a/dbms/src/Processors/ISource.cpp +++ b/dbms/src/Processors/ISource.cpp @@ -1,6 +1,5 @@ #include - namespace DB { diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 5048e510bbc..94abb2fc95e 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -577,9 +577,6 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( { /// Add columns needed to calculate the sorting expression and the sign. std::vector add_columns = data.sorting_key_expr->getRequiredColumns(); - std::cout << "std::vector add_columns" << std::endl; - for (auto column: add_columns) - std::cout << column << std::endl; column_names_to_read.insert(column_names_to_read.end(), add_columns.begin(), add_columns.end()); if (!data.merging_params.sign_column.empty()) @@ -1115,12 +1112,8 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( BlockInputStreams streams; streams.reserve(num_streams); - for (size_t i = 0; i < num_streams; ++i) { - std::cout << "labmda for loop № " << i << std::endl; - std::cout << pipes[i].getHeader().dumpStructure() << std::endl; + for (size_t i = 0; i < num_streams; ++i) streams.emplace_back(std::make_shared(std::move(pipes[i]))); - } - pipes.clear(); return streams; From a4189b85e29205ee76f3f0096822f1f8254d094a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 26 Nov 2019 17:57:33 +0300 Subject: [PATCH 18/77] done --- .../SummingSortedBlockInputStream.cpp | 7 +- ...01030_incorrect_count_merge_tree.reference | 9 +- .../01030_incorrect_count_merge_tree.sql | 6 +- ...correct_count_summing_merge_tree.reference | 78 +++++++++++++++++ ...030_incorrect_count_summing_merge_tree.sql | 84 ++++++++++++++++++- 5 files changed, 170 insertions(+), 14 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index 224475af636..acdaca508bd 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -78,7 +78,9 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( else { bool is_agg_func = WhichDataType(column.type).isAggregateFunction(); - if (!column.type->isSummable() && !is_agg_func) + + /// There are special const columns for example after prewere sections. + if ((!column.type->isSummable() && !is_agg_func) || isColumnConst(*column.column)) { column_numbers_not_to_aggregate.push_back(i); continue; @@ -198,9 +200,6 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns) { - if (columns_to_aggregate.empty()) - current_row_is_zero = false; - for (auto & desc : columns_to_aggregate) { // Do not insert if the aggregation state hasn't been created diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference index a5f040c71aa..487b1165348 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference @@ -1,5 +1,4 @@ -ClickHouse client version 19.17.1.1. -Connecting to localhost:9000 as user default. -Connected to ClickHouse server version 19.17.1 revision 54428. - -jakalletti :) \ No newline at end of file +2 +2 +2 +2 diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql index 95e40d4c377..7c496ab5ca6 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql @@ -8,9 +8,9 @@ create table tst2 insert into tst2 values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); -select * from tst2; +select count() from tst2; select count() from tst2 where val is not null; select count() from tst2 where timestamp is not null; -select count(*) from tst2 where timestamp > '2017-01-01 00:00:00' +select count() from tst2 where timestamp > '2017-01-01 00:00:00'; -drop table tst2; +drop table if exists tst2; diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference new file mode 100644 index 00000000000..6df56f0f19d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference @@ -0,0 +1,78 @@ +-- SummingMergeTree with Nullable column without duplicates. +2018-02-01 00:00:00 1 +2018-02-02 00:00:00 2 +-- 2 2 +2 +1 +-- 2 2 +2 +1 +-- 2 2 +2 +2 +-- 2 2 2 +1 +1 +1 +-- 2 2 +2 +1 +-- SummingMergeTree with Nullable column with duplicates +2018-02-01 00:00:00 4 +2018-02-02 00:00:00 6 +-- 4 2 +4 +2 +-- 4 2 +4 +2 +-- 4 2 +4 +2 +-- 2 2 2 2 +2 +2 +2 +2 +-- 2 2 +2 +2 +-- SummingMergeTree without Nullable column without duplicates. +2018-02-01 00:00:00 1 +2018-02-02 00:00:00 2 +-- 2 2 +2 +2 +-- 2 2 +2 +2 +-- 2 2 +2 +2 +-- 2 2 2 +1 +1 +1 +-- 2 2 +2 +2 +-- SummingMergeTree without Nullable column with duplicates. +2018-02-01 00:00:00 4 +2018-02-02 00:00:00 6 +-- 4 2 +4 +2 +-- 4 2 +4 +2 +-- 4 2 +4 +2 +-- 2 2 2 +2 +2 +2 +2 +-- 2 2 +2 +2 diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql index 770e24f7d1f..805af881480 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql @@ -1,40 +1,120 @@ +select '-- SummingMergeTree with Nullable column without duplicates.'; + drop table if exists tst; create table tst (timestamp DateTime, val Nullable(Int8)) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); +select * from tst final; + +select '-- 2 2'; select count() from tst; select count() from tst final; +select '-- 2 2'; select count() from tst where timestamp is not null; select count() from tst final where timestamp is not null; +select '-- 2 2'; select count() from tst where val is not null; select count() from tst final where val is not null; +select '-- 2 2 2'; select count() from tst final where timestamp>0; select count() from tst final prewhere timestamp > 0; -select count() from tst final where timestamp > '2017-01-01 00:00:00' +select count() from tst final where timestamp > '2017-01-01 00:00:00'; +select '-- 2 2'; select count() from tst final where val>0; select count() from tst final prewhere val>0; +select '-- SummingMergeTree with Nullable column with duplicates'; + +drop table if exists tst; +create table tst (timestamp DateTime, val Nullable(Int8)) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); +insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2), ('2018-02-01 00:00:00', 3), ('2018-02-02 00:00:00', 4); + +select * from tst final; + +select '-- 4 2'; +select count() from tst; +select count() from tst final; + +select '-- 4 2'; +select count() from tst where timestamp is not null; +select count() from tst final where timestamp is not null; + +select '-- 4 2'; +select count() from tst where val is not null; +select count() from tst final where val is not null; + +select '-- 2 2 2 2'; +select count() from tst final where timestamp>0; +select count() from tst final prewhere timestamp > 0; +select count() from tst final where timestamp > '2017-01-01 00:00:00'; +select count() from tst final prewhere timestamp > '2017-01-01 00:00:00'; + +select '-- 2 2'; +select count() from tst final where val>0; +select count() from tst final prewhere val>0; + +select '-- SummingMergeTree without Nullable column without duplicates.'; + drop table if exists tst; create table tst (timestamp DateTime, val Int8) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); +select * from tst final; + +select '-- 2 2'; select count() from tst; select count() from tst final; +select '-- 2 2 '; select count() from tst where timestamp is not null; select count() from tst final where timestamp is not null; +select '-- 2 2'; select count() from tst where val is not null; select count() from tst final where val is not null; +select '-- 2 2 2'; select count() from tst final where timestamp>0; select count() from tst final prewhere timestamp > 0; -select count() from tst final where timestamp > '2017-01-01 00:00:00' +select count() from tst final where timestamp > '2017-01-01 00:00:00'; +select '-- 2 2'; +select count() from tst final where val>0; +select count() from tst final prewhere val>0; + +drop table tst; + +select '-- SummingMergeTree without Nullable column with duplicates.'; + +drop table if exists tst; +create table tst (timestamp DateTime, val Int8) engine SummingMergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); +insert into tst values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2), ('2018-02-01 00:00:00', 3), ('2018-02-02 00:00:00', 4); + +select * from tst final; + +select '-- 4 2'; +select count() from tst; +select count() from tst final; + +select '-- 4 2'; +select count() from tst where timestamp is not null; +select count() from tst final where timestamp is not null; + +select '-- 4 2'; +select count() from tst where val is not null; +select count() from tst final where val is not null; + +select '-- 2 2 2'; +select count() from tst final where timestamp>0; +select count() from tst final prewhere timestamp > 0; +select count() from tst final where timestamp > '2017-01-01 00:00:00'; +select count() from tst final prewhere timestamp > '2017-01-01 00:00:00'; + +select '-- 2 2'; select count() from tst final where val>0; select count() from tst final prewhere val>0; From 013d839dcfe5159473af4df20aee0a5deb62082f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Tue, 26 Nov 2019 19:18:50 +0300 Subject: [PATCH 19/77] cleanup --- dbms/src/DataStreams/MergingSortedBlockInputStream.cpp | 1 - dbms/src/DataStreams/SummingSortedBlockInputStream.cpp | 2 -- dbms/src/Processors/ISource.cpp | 1 + dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- 4 files changed, 2 insertions(+), 4 deletions(-) diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp index 0c9055287a2..8c0707e09b0 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -24,7 +24,6 @@ MergingSortedBlockInputStream::MergingSortedBlockInputStream( , cursors(inputs_.size()), out_row_sources_buf(out_row_sources_buf_) { children.insert(children.end(), inputs_.begin(), inputs_.end()); - children.at(0)->dumpTree(std::cout, 0); header = children.at(0)->getHeader(); num_columns = header.columns(); } diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index acdaca508bd..810fc9576ce 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -272,7 +272,6 @@ Block SummingSortedBlockInputStream::readImpl() if (merged_columns.empty()) return {}; - /// Update aggregation result columns for current block for (auto & desc : columns_to_aggregate) { @@ -288,7 +287,6 @@ Block SummingSortedBlockInputStream::readImpl() } else desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty(); - } merge(merged_columns, queue_without_collation); diff --git a/dbms/src/Processors/ISource.cpp b/dbms/src/Processors/ISource.cpp index dcda869e1d4..d40f0e32fb7 100644 --- a/dbms/src/Processors/ISource.cpp +++ b/dbms/src/Processors/ISource.cpp @@ -1,5 +1,6 @@ #include + namespace DB { diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 94abb2fc95e..60ed25ed43c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1137,7 +1137,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( case MergeTreeData::MergingParams::Summing: merged = std::make_shared(streams_to_merge(), - sort_description, data.merging_params.columns_to_sum, max_block_size); + sort_description, data.merging_params.columns_to_sum, max_block_size); break; case MergeTreeData::MergingParams::Aggregating: From 0251f4e442f097247b4889788347829d56f3d84a Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 2 Dec 2019 14:45:21 +0300 Subject: [PATCH 20/77] perf: move applyLazyDefaults() out of cycle for ALL JOIN --- dbms/src/Interpreters/Join.cpp | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 7a217812710..2be7f58767a 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -677,9 +677,12 @@ private: template void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & added, IColumn::Offset & current_offset) { + if constexpr (add_missing) + added.applyLazyDefaults(); + for (auto it = mapped.begin(); it.ok(); ++it) { - added.appendFromBlock(*it->block, it->row_num); + added.appendFromBlock(*it->block, it->row_num); ++current_offset; } }; From 5ce65546e93cbb798e5ca0df708618be1728688a Mon Sep 17 00:00:00 2001 From: chertus Date: Mon, 2 Dec 2019 21:07:27 +0300 Subject: [PATCH 21/77] semi & anti join --- dbms/src/Interpreters/Join.cpp | 164 ++++++++---------- dbms/src/Interpreters/Join.h | 20 ++- dbms/src/Interpreters/joinDispatch.h | 18 +- dbms/src/Parsers/ASTTablesInSelectQuery.cpp | 6 + dbms/src/Parsers/ASTTablesInSelectQuery.h | 6 +- dbms/src/Parsers/ExpressionElementParsers.cpp | 5 +- .../src/Parsers/ParserTablesInSelectQuery.cpp | 14 +- dbms/src/Storages/StorageJoin.cpp | 6 +- ..._removing_unused_columns_from_subquery.sql | 2 +- .../0_stateless/00679_replace_asterisk.sql | 7 +- ...identical_result_after_merge_zookeeper.sql | 3 +- .../0_stateless/00859_distinct_with_join.sql | 4 +- ...00956_join_use_nulls_with_array_column.sql | 3 +- .../01009_insert_select_data_loss.sql | 2 +- .../01009_insert_select_nicelulu.sql | 3 +- .../0_stateless/01031_new_any_join.reference | 4 +- .../0_stateless/01031_new_any_join.sql | 2 +- .../01031_semi_anti_join.reference | 15 ++ .../0_stateless/01031_semi_anti_join.sql | 25 +++ 19 files changed, 191 insertions(+), 118 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01031_semi_anti_join.reference create mode 100644 dbms/tests/queries/0_stateless/01031_semi_anti_join.sql diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 2be7f58767a..d4f09415920 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -193,10 +193,10 @@ static const IColumn * extractAsofColumn(const ColumnRawPtrs & key_columns) return key_columns.back(); } -template +template static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes) { - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + if constexpr (is_asof_join) { auto key_column_copy = key_columns; auto key_size_copy = key_sizes; @@ -360,28 +360,19 @@ void Join::setSampleBlock(const Block & block) namespace { /// Inserting an element into a hash table of the form `key -> reference to a string`, which will then be used by JOIN. - template + template struct Inserter { - static void insert(const Join &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool); - }; - - template - struct Inserter - { - static ALWAYS_INLINE void insert(const Join & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE void insertOne(const Join & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, + Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); if (emplace_result.isInserted() || join.anyTakeLastRow()) new (&emplace_result.getMapped()) typename Map::mapped_type(stored_block, i); } - }; - template - struct Inserter - { - static ALWAYS_INLINE void insert(const Join &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) + static ALWAYS_INLINE void insertAll(const Join &, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool) { auto emplace_result = key_getter.emplaceKey(map, i, pool); @@ -393,13 +384,9 @@ namespace emplace_result.getMapped().insert({stored_block, i}, pool); } } - }; - template - struct Inserter - { - static ALWAYS_INLINE void insert(Join & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, - const IColumn * asof_column) + static ALWAYS_INLINE void insertAsof(Join & join, Map & map, KeyGetter & key_getter, Block * stored_block, size_t i, Arena & pool, + const IColumn * asof_column) { auto emplace_result = key_getter.emplaceKey(map, i, pool); typename Map::mapped_type * time_series_map = &emplace_result.getMapped(); @@ -416,30 +403,27 @@ namespace Join & join, Map & map, size_t rows, const ColumnRawPtrs & key_columns, const Sizes & key_sizes, Block * stored_block, ConstNullMapPtr null_map, Arena & pool) { + constexpr bool mapped_one = std::is_same_v || + std::is_same_v; + constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; + const IColumn * asof_column [[maybe_unused]] = nullptr; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + if constexpr (is_asof_join) asof_column = extractAsofColumn(key_columns); - auto key_getter = createKeyGetter(key_columns, key_sizes); + auto key_getter = createKeyGetter(key_columns, key_sizes); for (size_t i = 0; i < rows; ++i) { if (has_null_map && (*null_map)[i]) continue; - if constexpr (STRICTNESS == ASTTableJoin::Strictness::Any) - { - constexpr bool mapped_one = std::is_same_v || - std::is_same_v; - if constexpr (mapped_one) - Inserter::insert(join, map, key_getter, stored_block, i, pool); - else - Inserter::insert(join, map, key_getter, stored_block, i, pool); - } - else if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) - Inserter::insert(join, map, key_getter, stored_block, i, pool, asof_column); + if constexpr (is_asof_join) + Inserter::insertAsof(join, map, key_getter, stored_block, i, pool, asof_column); + else if constexpr (mapped_one) + Inserter::insertOne(join, map, key_getter, stored_block, i, pool); else - Inserter::insert(join, map, key_getter, stored_block, i, pool); + Inserter::insertAll(join, map, key_getter, stored_block, i, pool); } } @@ -706,11 +690,14 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; - constexpr bool left_or_full = static_in_v; + constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi; + constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti; + constexpr bool left = KIND == ASTTableJoin::Kind::Left; constexpr bool right = KIND == ASTTableJoin::Kind::Right; + constexpr bool full = KIND == ASTTableJoin::Kind::Full; - constexpr bool add_missing = left_or_full; - constexpr bool need_replication = is_all_join || (is_any_join && right); + constexpr bool add_missing = (left || full) && !is_semi_join; + constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right); size_t rows = added_columns.rows_to_add; IColumn::Filter filter(rows, 0); @@ -723,7 +710,7 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added if constexpr (is_asof_join) asof_column = extractAsofColumn(added_columns.key_columns); - auto key_getter = createKeyGetter(added_columns.key_columns, added_columns.key_sizes); + auto key_getter = createKeyGetter(added_columns.key_columns, added_columns.key_sizes); IColumn::Offset current_offset = 0; @@ -768,7 +755,7 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added mapped.setUsed(); addFoundRowAll(mapped, added_columns, current_offset); } - else if constexpr (is_any_join && right) + else if constexpr ((is_any_join || is_semi_join) && right) { /// Use first appered left key + it needs left columns replication if (mapped.setUsedOnce()) @@ -786,11 +773,16 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added added_columns.appendFromBlock(*mapped.block, mapped.row_num); } } - else if constexpr (is_any_join && KIND == ASTTableJoin::Kind::Full) + else if constexpr (is_any_join && full) { /// TODO } - else /// ANY LEFT + old ANY (RightAny) + else if constexpr (is_anti_join) + { + if constexpr (right) + mapped.setUsed(); + } + else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) { filter[i] = 1; mapped.setUsed(); @@ -798,7 +790,11 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added } } else + { + if constexpr (is_anti_join && left) + filter[i] = 1; addNotFoundRow(added_columns, current_offset); + } if constexpr (need_replication) (*added_columns.offsets_to_replicate)[i] = current_offset; @@ -849,12 +845,16 @@ void Join::joinBlockImpl( constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; constexpr bool is_all_join = STRICTNESS == ASTTableJoin::Strictness::All; constexpr bool is_asof_join = STRICTNESS == ASTTableJoin::Strictness::Asof; - constexpr bool right = KIND == ASTTableJoin::Kind::Right; - constexpr bool inner_or_right = static_in_v; - constexpr bool right_or_full = static_in_v; + constexpr bool is_semi_join = STRICTNESS == ASTTableJoin::Strictness::Semi; + constexpr bool is_anti_join = STRICTNESS == ASTTableJoin::Strictness::Anti; - constexpr bool need_filter = (!is_all_join && inner_or_right) && !(is_any_join && right); - constexpr bool need_replication = is_all_join || (is_any_join && right); + constexpr bool left = KIND == ASTTableJoin::Kind::Left; + constexpr bool right = KIND == ASTTableJoin::Kind::Right; + constexpr bool inner = KIND == ASTTableJoin::Kind::Inner; + constexpr bool full = KIND == ASTTableJoin::Kind::Full; + + constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right); + constexpr bool need_filter = !need_replication && (inner || right || (is_semi_join && left) || (is_anti_join && left)); /// Rare case, when keys are constant. To avoid code bloat, simply materialize them. Columns materialized_columns; @@ -870,7 +870,7 @@ void Join::joinBlockImpl( * Because if they are constants, then in the "not joined" rows, they may have different values * - default values, which can differ from the values of these constants. */ - if constexpr (right_or_full) + if constexpr (right || full) { materializeBlockInplace(block); @@ -1085,61 +1085,44 @@ void Join::joinTotals(Block & block) const } -template -struct AdderNonJoined; - template -struct AdderNonJoined -{ - static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right) - { - for (size_t j = 0; j < columns_right.size(); ++j) - { - const auto & mapped_column = mapped.block->getByPosition(j).column; - columns_right[j]->insertFrom(*mapped_column, mapped.row_num); - } - - ++rows_added; - } -}; - -template -struct AdderNonJoined +struct AdderNonJoined { static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right) { + constexpr bool mapped_asof = std::is_same_v; constexpr bool mapped_one = std::is_same_v || std::is_same_v; - if constexpr (!mapped_one) - AdderNonJoined::add(mapped, rows_added, columns_right); - } -}; -template -struct AdderNonJoined -{ - static void add(const Mapped & mapped, size_t & rows_added, MutableColumns & columns_right) - { - for (auto it = mapped.begin(); it.ok(); ++it) + if constexpr (mapped_asof) + { + /// Do nothing + } + else if constexpr (mapped_one) { for (size_t j = 0; j < columns_right.size(); ++j) { - const auto & mapped_column = it->block->getByPosition(j).column; - columns_right[j]->insertFrom(*mapped_column, it->row_num); + const auto & mapped_column = mapped.block->getByPosition(j).column; + columns_right[j]->insertFrom(*mapped_column, mapped.row_num); } ++rows_added; } + else + { + for (auto it = mapped.begin(); it.ok(); ++it) + { + for (size_t j = 0; j < columns_right.size(); ++j) + { + const auto & mapped_column = it->block->getByPosition(j).column; + columns_right[j]->insertFrom(*mapped_column, it->row_num); + } + + ++rows_added; + } + } } }; -template -struct AdderNonJoined -{ - static void add(const Mapped & /*mapped*/, size_t & /*rows_added*/, MutableColumns & /*columns_right*/) - { - // If we have a leftover match in the right hand side, not required to join because we are only support asof left/inner - } -}; /// Stream from not joined earlier rows of the right table. class NonJoinedBlockInputStream : public IBlockInputStream @@ -1348,10 +1331,11 @@ private: for (; it != end; ++it) { const Mapped & mapped = it->getMapped(); + if (mapped.getUsed()) continue; - AdderNonJoined::add(mapped, rows_added, columns_keys_and_right); + AdderNonJoined::add(mapped, rows_added, columns_keys_and_right); if (rows_added >= max_block_size) { @@ -1391,6 +1375,10 @@ private: BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & result_sample_block, UInt64 max_block_size) const { + if (table_join->strictness() == ASTTableJoin::Strictness::Asof || + table_join->strictness() == ASTTableJoin::Strictness::Semi) + return {}; + if (isRightOrFull(table_join->kind())) return std::make_shared(*this, result_sample_block, max_block_size); return {}; diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 58ae04f25d8..11e48c6908e 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -80,11 +80,23 @@ using MappedAsof = WithFlags; * It is just a hash table: keys -> rows of joined ("right") table. * Additionally, CROSS JOIN is supported: instead of hash table, it use just set of blocks without keys. * - * JOIN-s could be of nine types: ANY/ALL × LEFT/INNER/RIGHT/FULL, and also CROSS. + * JOIN-s could be of these types: + * - ALL × LEFT/INNER/RIGHT/FULL + * - ANY × LEFT/INNER/RIGHT + * - SEMI/ANTI x LEFT/RIGHT + * - ASOF x LEFT/INNER + * - CROSS * - * If ANY is specified - then select only one row from the "right" table, (first encountered row), even if there was more matching rows. - * If ALL is specified - usual JOIN, when rows are multiplied by number of matching rows from the "right" table. - * ANY is more efficient. + * ALL means usual JOIN, when rows are multiplied by number of matching rows from the "right" table. + * ANY uses one line per unique key from right talbe. For LEFT JOIN it would be any row (with needed joined key) from the right table, + * for RIGHT JOIN it would be any row from the left table and for INNER one it would be any row from right and any row from left. + * SEMI JOIN filter left table by keys that are present in right table for LEFT JOIN, and filter right table by keys from left table + * for RIGHT JOIN. In other words SEMI JOIN returns only rows which joining keys present in another table. + * ANTI JOIN is the same as SEMI JOIN but returns rows with joining keys that are NOT present in another table. + * SEMI/ANTI JOINs allow to get values from both tables. For filter table it gets any row with joining same key. For ANTI JOIN it returns + * defaults other table columns. + * ASOF JOIN is not-equi join. For one key column it finds nearest value to join according to join inequality. + * It's expected that ANY|SEMI LEFT JOIN is more efficient that ALL one. * * If INNER is specified - leave only rows that have matching rows from "right" table. * If LEFT is specified - in case when there is no matching row in "right" table, fill it with default values instead. diff --git a/dbms/src/Interpreters/joinDispatch.h b/dbms/src/Interpreters/joinDispatch.h index e5bb644f504..593a19ab637 100644 --- a/dbms/src/Interpreters/joinDispatch.h +++ b/dbms/src/Interpreters/joinDispatch.h @@ -30,6 +30,18 @@ template <> struct MapGetter struct MapGetter { using Map = Join::MapsAllFlagged; }; template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; +/// Only SEMI LEFT and SEMI RIGHT are valid +template <> struct MapGetter { using Map = Join::MapsOne; }; +template <> struct MapGetter { using Map = Join::MapsOne; }; +template <> struct MapGetter { using Map = Join::MapsAll; }; +template <> struct MapGetter { using Map = Join::MapsOne; }; + +/// Only ANTI LEFT and ANTI RIGHT are valid +template <> struct MapGetter { using Map = Join::MapsOne; }; +template <> struct MapGetter { using Map = Join::MapsOne; }; +template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; +template <> struct MapGetter { using Map = Join::MapsOne; }; + template struct MapGetter { @@ -37,11 +49,13 @@ struct MapGetter }; -static constexpr std::array STRICTNESSES = { +static constexpr std::array STRICTNESSES = { ASTTableJoin::Strictness::RightAny, ASTTableJoin::Strictness::Any, ASTTableJoin::Strictness::All, - ASTTableJoin::Strictness::Asof + ASTTableJoin::Strictness::Asof, + ASTTableJoin::Strictness::Semi, + ASTTableJoin::Strictness::Anti, }; static constexpr std::array KINDS = { diff --git a/dbms/src/Parsers/ASTTablesInSelectQuery.cpp b/dbms/src/Parsers/ASTTablesInSelectQuery.cpp index 9fc130adb31..18750d5ccd3 100644 --- a/dbms/src/Parsers/ASTTablesInSelectQuery.cpp +++ b/dbms/src/Parsers/ASTTablesInSelectQuery.cpp @@ -150,6 +150,12 @@ void ASTTableJoin::formatImplBeforeTable(const FormatSettings & settings, Format case Strictness::Asof: settings.ostr << "ASOF "; break; + case Strictness::Semi: + settings.ostr << "SEMI "; + break; + case Strictness::Anti: + settings.ostr << "ANTI "; + break; } } diff --git a/dbms/src/Parsers/ASTTablesInSelectQuery.h b/dbms/src/Parsers/ASTTablesInSelectQuery.h index 57ba6ca93f2..01c6914b46c 100644 --- a/dbms/src/Parsers/ASTTablesInSelectQuery.h +++ b/dbms/src/Parsers/ASTTablesInSelectQuery.h @@ -25,7 +25,7 @@ namespace DB * SAMPLE 1000000 * * Table expressions may be combined with JOINs of following kinds: - * [GLOBAL] [ANY|ALL|] INNER|LEFT|RIGHT|FULL [OUTER] JOIN table_expr + * [GLOBAL] [ANY|ALL|ASOF|SEMI] [INNER|LEFT|RIGHT|FULL] [OUTER] JOIN table_expr * CROSS JOIN * , (comma) * @@ -74,10 +74,12 @@ struct ASTTableJoin : public IAST enum class Strictness { Unspecified, - RightAny, /// Right ANY. If there are many suitable rows in right table, use any from them to join. + RightAny, /// Old ANY JOIN. If there are many suitable rows in right table, use any from them to join. Any, /// Semi Join with any value from filtering table. For LEFT JOIN with Any and RightAny are the same. All, /// If there are many suitable rows to join, use all of them and replicate rows of "left" table (usual semantic of JOIN). Asof, /// For the last JOIN column, pick the latest value + Semi, /// LEFT or RIGHT. SEMI LEFT JOIN filters left table by values exists in right table. SEMI RIGHT - otherwise. + Anti, /// LEFT or RIGHT. Same as SEMI JOIN but filter values that are NOT exists in other table. }; /// Join method. diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 89793a5042d..99dce13ef76 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -1134,11 +1134,14 @@ const char * ParserAlias::restricted_keywords[] = "INNER", "FULL", "CROSS", - "ASOF", "JOIN", "GLOBAL", "ANY", "ALL", + "ASOF", + "SEMI", + "ANTI", + "ONLY", /// YQL synonym for ANTI "ON", "USING", "PREWHERE", diff --git a/dbms/src/Parsers/ParserTablesInSelectQuery.cpp b/dbms/src/Parsers/ParserTablesInSelectQuery.cpp index 6b970b0565f..7e84925b203 100644 --- a/dbms/src/Parsers/ParserTablesInSelectQuery.cpp +++ b/dbms/src/Parsers/ParserTablesInSelectQuery.cpp @@ -137,6 +137,10 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec table_join->strictness = ASTTableJoin::Strictness::All; else if (ParserKeyword("ASOF").ignore(pos)) table_join->strictness = ASTTableJoin::Strictness::Asof; + else if (ParserKeyword("SEMI").ignore(pos)) + table_join->strictness = ASTTableJoin::Strictness::Semi; + else if (ParserKeyword("ANTI").ignore(pos) || ParserKeyword("ONLY").ignore(pos)) + table_join->strictness = ASTTableJoin::Strictness::Anti; else table_join->strictness = ASTTableJoin::Strictness::Unspecified; @@ -153,13 +157,21 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec else { /// Use INNER by default as in another DBMS. - table_join->kind = ASTTableJoin::Kind::Inner; + if (table_join->strictness == ASTTableJoin::Strictness::Semi || + table_join->strictness == ASTTableJoin::Strictness::Anti) + table_join->kind = ASTTableJoin::Kind::Left; + else + table_join->kind = ASTTableJoin::Kind::Inner; } if (table_join->strictness != ASTTableJoin::Strictness::Unspecified && table_join->kind == ASTTableJoin::Kind::Cross) throw Exception("You must not specify ANY or ALL for CROSS JOIN.", ErrorCodes::SYNTAX_ERROR); + if ((table_join->strictness == ASTTableJoin::Strictness::Semi || table_join->strictness == ASTTableJoin::Strictness::Anti) && + (table_join->kind != ASTTableJoin::Kind::Left && table_join->kind != ASTTableJoin::Kind::Right)) + throw Exception("SEMI|ANTI JOIN should be LEFT or RIGHT.", ErrorCodes::SYNTAX_ERROR); + /// Optional OUTER keyword for outer joins. if (table_join->kind == ASTTableJoin::Kind::Left || table_join->kind == ASTTableJoin::Kind::Right diff --git a/dbms/src/Storages/StorageJoin.cpp b/dbms/src/Storages/StorageJoin.cpp index 4ec8741df2f..12444867b6b 100644 --- a/dbms/src/Storages/StorageJoin.cpp +++ b/dbms/src/Storages/StorageJoin.cpp @@ -343,9 +343,11 @@ private: throw Exception("New ANY join storage is not implemented yet (set any_join_distinct_right_table_keys=1 to use old one)", ErrorCodes::NOT_IMPLEMENTED); } - else if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof) + else if constexpr (STRICTNESS == ASTTableJoin::Strictness::Asof || + STRICTNESS == ASTTableJoin::Strictness::Semi || + STRICTNESS == ASTTableJoin::Strictness::Anti) { - throw Exception("ASOF join storage is not implemented yet", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("ASOF|SEMI|ANTI join storage is not implemented yet", ErrorCodes::NOT_IMPLEMENTED); } else for (auto ref_it = it->getMapped().begin(); ref_it.ok(); ++ref_it) diff --git a/dbms/tests/queries/0_stateless/00586_removing_unused_columns_from_subquery.sql b/dbms/tests/queries/0_stateless/00586_removing_unused_columns_from_subquery.sql index c22b72e4126..ce52c652df0 100644 --- a/dbms/tests/queries/0_stateless/00586_removing_unused_columns_from_subquery.sql +++ b/dbms/tests/queries/0_stateless/00586_removing_unused_columns_from_subquery.sql @@ -20,7 +20,7 @@ FROM learnerHash, passed - eventTime AS diff FROM statements - GLOBAL ANY INNER JOIN + GLOBAL SEMI LEFT JOIN ( SELECT learnerHash, diff --git a/dbms/tests/queries/0_stateless/00679_replace_asterisk.sql b/dbms/tests/queries/0_stateless/00679_replace_asterisk.sql index 27ff799be62..19aa939b132 100644 --- a/dbms/tests/queries/0_stateless/00679_replace_asterisk.sql +++ b/dbms/tests/queries/0_stateless/00679_replace_asterisk.sql @@ -1,7 +1,6 @@ -set any_join_distinct_right_table_keys = 1; SET joined_subquery_requires_alias = 0; SELECT * FROM (SELECT 1 AS id, 2 AS value); -SELECT * FROM (SELECT 1 AS id, 2 AS value, 3 AS A) ANY INNER JOIN (SELECT 1 AS id, 4 AS values, 5 AS D) USING id; -SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) ANY INNER JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; -SELECT *, d.*, d.values FROM ( SELECT 1 AS id, 2 AS value ) ANY INNER JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; +SELECT * FROM (SELECT 1 AS id, 2 AS value, 3 AS A) SEMI LEFT JOIN (SELECT 1 AS id, 4 AS values, 5 AS D) USING id; +SELECT *, d.* FROM ( SELECT 1 AS id, 2 AS value ) SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; +SELECT *, d.*, d.values FROM ( SELECT 1 AS id, 2 AS value ) SEMI LEFT JOIN ( SELECT 1 AS id, 3 AS values ) AS d USING id; diff --git a/dbms/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql b/dbms/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql index abf2903d3ea..aa386829276 100644 --- a/dbms/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00721_force_by_identical_result_after_merge_zookeeper.sql @@ -12,8 +12,7 @@ SYSTEM SYNC REPLICA byte_identical_r2; ALTER TABLE byte_identical_r1 ADD COLUMN y DEFAULT rand(); OPTIMIZE TABLE byte_identical_r1 PARTITION tuple() FINAL; -SET any_join_distinct_right_table_keys = 1; -SELECT x, t1.y - t2.y FROM byte_identical_r1 t1 ANY INNER JOIN byte_identical_r2 t2 USING x ORDER BY x; +SELECT x, t1.y - t2.y FROM byte_identical_r1 t1 SEMI LEFT JOIN byte_identical_r2 t2 USING x ORDER BY x; DROP TABLE byte_identical_r1; DROP TABLE byte_identical_r2; diff --git a/dbms/tests/queries/0_stateless/00859_distinct_with_join.sql b/dbms/tests/queries/0_stateless/00859_distinct_with_join.sql index 23c41549502..4fb6f4ec046 100644 --- a/dbms/tests/queries/0_stateless/00859_distinct_with_join.sql +++ b/dbms/tests/queries/0_stateless/00859_distinct_with_join.sql @@ -1,5 +1,3 @@ -set any_join_distinct_right_table_keys = 1; - drop table if exists fooL; drop table if exists fooR; create table fooL (a Int32, v String) engine = Memory; @@ -9,7 +7,7 @@ insert into fooL select number, 'L' || toString(number) from numbers(2); insert into fooL select number, 'LL' || toString(number) from numbers(2); insert into fooR select number, 'R' || toString(number) from numbers(2); -select distinct a from fooL any join fooR using(a) order by a; +select distinct a from fooL semi left join fooR using(a) order by a; drop table fooL; drop table fooR; diff --git a/dbms/tests/queries/0_stateless/00956_join_use_nulls_with_array_column.sql b/dbms/tests/queries/0_stateless/00956_join_use_nulls_with_array_column.sql index 244e04a564a..f70bccd68fd 100644 --- a/dbms/tests/queries/0_stateless/00956_join_use_nulls_with_array_column.sql +++ b/dbms/tests/queries/0_stateless/00956_join_use_nulls_with_array_column.sql @@ -1,4 +1,3 @@ -SET any_join_distinct_right_table_keys = 1; SET join_use_nulls = 1; -SELECT number FROM system.numbers ANY INNER JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) js2 USING (number) LIMIT 1; +SELECT number FROM system.numbers SEMI LEFT JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) js2 USING (number) LIMIT 1; SELECT number FROM system.numbers ANY LEFT JOIN (SELECT number, ['test'] FROM system.numbers LIMIT 1) js2 USING (number) LIMIT 1; diff --git a/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql b/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql index 9a754d94323..7ecffd8653c 100644 --- a/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql +++ b/dbms/tests/queries/0_stateless/01009_insert_select_data_loss.sql @@ -1,5 +1,5 @@ drop table if exists tab; create table tab (x UInt64) engine = MergeTree order by tuple(); -insert into tab select number as n from numbers(20) any inner join (select number * 10 as n from numbers(2)) using(n) settings any_join_distinct_right_table_keys = 1, max_block_size = 5; +insert into tab select number as n from numbers(20) semi left join (select number * 10 as n from numbers(2)) using(n) settings max_block_size = 5; select * from tab order by x; diff --git a/dbms/tests/queries/0_stateless/01009_insert_select_nicelulu.sql b/dbms/tests/queries/0_stateless/01009_insert_select_nicelulu.sql index 90a902c352d..3fe7ec04e85 100644 --- a/dbms/tests/queries/0_stateless/01009_insert_select_nicelulu.sql +++ b/dbms/tests/queries/0_stateless/01009_insert_select_nicelulu.sql @@ -1,4 +1,3 @@ -Set any_join_distinct_right_table_keys=1; DROP TABLE IF EXISTS test_insert_t1; DROP TABLE IF EXISTS test_insert_t2; DROP TABLE IF EXISTS test_insert_t3; @@ -15,7 +14,7 @@ INSERT INTO test_insert_t2 SELECT '2019-09-01',toString(number) FROM system.numb INSERT INTO test_insert_t2 SELECT '2019-09-01',toString(number) FROM system.numbers WHERE number >=700000 limit 200; INSERT INTO test_insert_t2 SELECT '2019-09-01',toString(number) FROM system.numbers WHERE number >=900000 limit 200; -INSERT INTO test_insert_t3 SELECT '2019-09-01', uid, name, city FROM ( SELECT dt, uid, name, city FROM test_insert_t1 WHERE dt = '2019-09-01') t1 GLOBAL ANY INNER JOIN (SELECT uid FROM test_insert_t2 WHERE dt = '2019-09-01') t2 ON t1.uid=t2.uid; +INSERT INTO test_insert_t3 SELECT '2019-09-01', uid, name, city FROM ( SELECT dt, uid, name, city FROM test_insert_t1 WHERE dt = '2019-09-01') t1 GLOBAL SEMI LEFT JOIN (SELECT uid FROM test_insert_t2 WHERE dt = '2019-09-01') t2 ON t1.uid=t2.uid; SELECT count(*) FROM test_insert_t3; diff --git a/dbms/tests/queries/0_stateless/01031_new_any_join.reference b/dbms/tests/queries/0_stateless/01031_new_any_join.reference index f2e2119503d..1fd9a5352e3 100644 --- a/dbms/tests/queries/0_stateless/01031_new_any_join.reference +++ b/dbms/tests/queries/0_stateless/01031_new_any_join.reference @@ -5,12 +5,12 @@ any left 3 a4 0 4 a5 4 b3 any left (rev) +0 5 b6 2 a3 2 b1 2 a3 2 b2 4 a5 4 b3 4 a5 4 b4 4 a5 4 b5 -4 a5 4 b6 any inner 2 a3 2 b1 4 a5 4 b3 @@ -18,12 +18,12 @@ any inner (rev) 2 a3 2 b1 4 a5 4 b3 any right +0 5 b6 2 a3 2 b1 2 a3 2 b2 4 a5 4 b3 4 a5 4 b4 4 a5 4 b5 -4 a5 4 b6 any right (rev) 0 a1 0 1 a2 0 diff --git a/dbms/tests/queries/0_stateless/01031_new_any_join.sql b/dbms/tests/queries/0_stateless/01031_new_any_join.sql index 822895c520f..de86d8eebc5 100644 --- a/dbms/tests/queries/0_stateless/01031_new_any_join.sql +++ b/dbms/tests/queries/0_stateless/01031_new_any_join.sql @@ -5,7 +5,7 @@ CREATE TABLE t1 (x UInt32, s String) engine = Memory; CREATE TABLE t2 (x UInt32, s String) engine = Memory; INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'); -INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (4, 'b6'); +INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (5, 'b6'); SET join_use_nulls = 0; SET any_join_distinct_right_table_keys = 0; diff --git a/dbms/tests/queries/0_stateless/01031_semi_anti_join.reference b/dbms/tests/queries/0_stateless/01031_semi_anti_join.reference new file mode 100644 index 00000000000..5dbe67e50b4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01031_semi_anti_join.reference @@ -0,0 +1,15 @@ +semi left +2 a3 2 b1 +4 a5 4 b3 +semi right +2 a3 2 b1 +2 a3 2 b2 +4 a5 4 b3 +4 a5 4 b4 +4 a5 4 b5 +anti left +0 a1 0 +1 a2 1 +3 a4 3 +anti right +0 5 b6 diff --git a/dbms/tests/queries/0_stateless/01031_semi_anti_join.sql b/dbms/tests/queries/0_stateless/01031_semi_anti_join.sql new file mode 100644 index 00000000000..ee3b81834df --- /dev/null +++ b/dbms/tests/queries/0_stateless/01031_semi_anti_join.sql @@ -0,0 +1,25 @@ +DROP TABLE IF EXISTS t1; +DROP TABLE IF EXISTS t2; + +CREATE TABLE t1 (x UInt32, s String) engine = Memory; +CREATE TABLE t2 (x UInt32, s String) engine = Memory; + +INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'); +INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (5, 'b6'); + +SET join_use_nulls = 0; + +SELECT 'semi left'; +SELECT t1.*, t2.* FROM t1 SEMI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'semi right'; +SELECT t1.*, t2.* FROM t1 SEMI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'anti left'; +SELECT t1.*, t2.* FROM t1 ANTI LEFT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +SELECT 'anti right'; +SELECT t1.*, t2.* FROM t1 ANTI RIGHT JOIN t2 USING(x) ORDER BY t1.x, t2.x; + +DROP TABLE t1; +DROP TABLE t2; From 3dc6ef421ec4d0868de79b133ad9c8140de66de9 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 3 Dec 2019 15:55:20 +0300 Subject: [PATCH 22/77] perf: do not make row filter if not needed --- dbms/src/Interpreters/Join.cpp | 64 +++++++++++++++++++++++----------- 1 file changed, 43 insertions(+), 21 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index d4f09415920..2297b47b55d 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -583,6 +583,7 @@ public: , key_columns(key_columns_) , key_sizes(key_sizes_) , rows_to_add(block.rows()) + , need_filter(false) { size_t num_columns_to_add = sample_block_with_columns_to_add.columns(); @@ -643,6 +644,7 @@ public: const Sizes & key_sizes; size_t rows_to_add; std::unique_ptr offsets_to_replicate; + bool need_filter; private: TypeAndNames type_name; @@ -671,20 +673,28 @@ void addFoundRowAll(const typename Map::mapped_type & mapped, AddedColumns & add } }; -template +template void addNotFoundRow(AddedColumns & added [[maybe_unused]], IColumn::Offset & current_offset [[maybe_unused]]) { if constexpr (add_missing) { added.appendDefaultRow(); - ++current_offset; + if constexpr (need_offset) + ++current_offset; } } +template +void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]) +{ + if constexpr (need_filter) + filter[pos] = 1; +} + /// Joins right table columns which indexes are present in right_indexes using specified map. /// Makes filter (1 if row presented in right table) and returns offsets to replicate (for ALL JOINS). -template +template NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added_columns, const ConstNullMapPtr & null_map [[maybe_unused]]) { constexpr bool is_any_join = STRICTNESS == ASTTableJoin::Strictness::Any; @@ -700,7 +710,10 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added constexpr bool need_replication = is_all_join || (is_any_join && right) || (is_semi_join && right); size_t rows = added_columns.rows_to_add; - IColumn::Filter filter(rows, 0); + IColumn::Filter filter; + if constexpr (need_filter) + filter = IColumn::Filter(rows, 0); + Arena pool; if constexpr (need_replication) @@ -716,11 +729,11 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added for (size_t i = 0; i < rows; ++i) { - if constexpr (_has_null_map) + if constexpr (has_null_map) { if ((*null_map)[i]) { - addNotFoundRow(added_columns, current_offset); + addNotFoundRow(added_columns, current_offset); if constexpr (need_replication) (*added_columns.offsets_to_replicate)[i] = current_offset; @@ -739,19 +752,16 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added const Join & join = added_columns.join; if (const RowRef * found = mapped.findAsof(join.getAsofType(), join.getAsofInequality(), asof_column, i)) { - filter[i] = 1; + setUsed(filter, i); mapped.setUsed(); added_columns.appendFromBlock(*found->block, found->row_num); } else - { - filter[i] = 0; - addNotFoundRow(added_columns, current_offset); - } + addNotFoundRow(added_columns, current_offset); } else if constexpr (is_all_join) { - filter[i] = 1; + setUsed(filter, i); mapped.setUsed(); addFoundRowAll(mapped, added_columns, current_offset); } @@ -760,7 +770,7 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added /// Use first appered left key + it needs left columns replication if (mapped.setUsedOnce()) { - filter[i] = 1; + setUsed(filter, i); addFoundRowAll(mapped, added_columns, current_offset); } } @@ -769,7 +779,7 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added /// Use first appered left key only if (mapped.setUsedOnce()) { - filter[i] = 1; + setUsed(filter, i); added_columns.appendFromBlock(*mapped.block, mapped.row_num); } } @@ -784,7 +794,7 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added } else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) { - filter[i] = 1; + setUsed(filter, i); mapped.setUsed(); added_columns.appendFromBlock(*mapped.block, mapped.row_num); } @@ -792,8 +802,8 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added else { if constexpr (is_anti_join && left) - filter[i] = 1; - addNotFoundRow(added_columns, current_offset); + setUsed(filter, i); + addNotFoundRow(added_columns, current_offset); } if constexpr (need_replication) @@ -807,10 +817,20 @@ NO_INLINE IColumn::Filter joinRightColumns(const Map & map, AddedColumns & added template IColumn::Filter joinRightColumnsSwitchNullability(const Map & map, AddedColumns & added_columns, const ConstNullMapPtr & null_map) { - if (null_map) - return joinRightColumns(map, added_columns, null_map); + if (added_columns.need_filter) + { + if (null_map) + return joinRightColumns(map, added_columns, null_map); + else + return joinRightColumns(map, added_columns, nullptr); + } else - return joinRightColumns(map, added_columns, nullptr); + { + if (null_map) + return joinRightColumns(map, added_columns, null_map); + else + return joinRightColumns(map, added_columns, nullptr); + } } template @@ -889,6 +909,8 @@ void Join::joinBlockImpl( AddedColumns added_columns(sample_block_with_columns_to_add, block_with_columns_to_add, block, saved_block_sample, extras, *this, key_columns, key_sizes); + bool has_required_right_keys = (required_right_keys.columns() != 0); + added_columns.need_filter = need_filter || has_required_right_keys; IColumn::Filter row_filter = switchJoinRightColumns(maps_, added_columns, type, null_map); @@ -914,7 +936,7 @@ void Join::joinBlockImpl( block.insert(correctNullability({col.column, col.type, right_key.name}, is_nullable)); } } - else + else if (has_required_right_keys) { /// Some trash to represent IColumn::Filter as ColumnUInt8 needed for ColumnNullable::applyNullMap() auto null_map_filter_ptr = ColumnUInt8::create(); From 46b27e4dd1c08d5348ef8d0c1362e2d464768d36 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 3 Dec 2019 16:31:52 +0300 Subject: [PATCH 23/77] perf: move heavy addJoinedBlock() operations out of lock --- dbms/src/Interpreters/Join.cpp | 16 +++++++++------- dbms/src/Interpreters/Join.h | 2 +- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 2297b47b55d..7320572e962 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -501,7 +501,7 @@ void Join::initRightBlockStructure() JoinCommon::convertColumnsToNullable(saved_block_sample, (isFull(kind) ? right_table_keys.columns() : 0)); } -Block * Join::storeRightBlock(const Block & source_block) +Block Join::structureRightBlock(const Block & source_block) const { /// Rare case, when joined columns are constant. To avoid code bloat, simply materialize them. Block block = materializeBlock(source_block); @@ -515,14 +515,11 @@ Block * Join::storeRightBlock(const Block & source_block) structured_block.insert(column); } - blocks.push_back(structured_block); - return &blocks.back(); + return structured_block; } bool Join::addJoinedBlock(const Block & block) { - std::unique_lock lock(rwlock); - if (empty()) throw Exception("Logical error: Join was not initialized", ErrorCodes::LOGICAL_ERROR); @@ -534,12 +531,17 @@ bool Join::addJoinedBlock(const Block & block) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); + Block structured_block = structureRightBlock(block); + + std::unique_lock lock(rwlock); + + blocks.emplace_back(std::move(structured_block)); + Block * stored_block = &blocks.back(); + size_t rows = block.rows(); if (rows) has_no_rows_in_maps = false; - Block * stored_block = storeRightBlock(block); - if (kind != ASTTableJoin::Kind::Cross) { joinDispatch(kind, strictness, maps, [&](auto, auto strictness_, auto & map) diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 665fc0d6674..6342bd891a8 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -365,7 +365,7 @@ private: void setSampleBlock(const Block & block); /// Modify (structure) and save right block, @returns pointer to saved block - Block * storeRightBlock(const Block & stored_block); + Block structureRightBlock(const Block & stored_block) const; void initRightBlockStructure(); void initRequiredRightKeys(); From e160d35e549676b1317893ab396a6976c677dafe Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 3 Dec 2019 17:30:51 +0300 Subject: [PATCH 24/77] more Join.addJoinedBlock() refactoring --- dbms/src/Interpreters/Join.cpp | 60 +++++++++++++++++++--------------- 1 file changed, 34 insertions(+), 26 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 7320572e962..df0afee3eb9 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -531,37 +531,45 @@ bool Join::addJoinedBlock(const Block & block) ConstNullMapPtr null_map{}; ColumnPtr null_map_holder = extractNestedColumnsAndNullMap(key_columns, null_map); - Block structured_block = structureRightBlock(block); - - std::unique_lock lock(rwlock); - - blocks.emplace_back(std::move(structured_block)); - Block * stored_block = &blocks.back(); - - size_t rows = block.rows(); - if (rows) - has_no_rows_in_maps = false; - - if (kind != ASTTableJoin::Kind::Cross) - { - joinDispatch(kind, strictness, maps, [&](auto, auto strictness_, auto & map) - { - insertFromBlockImpl(*this, type, map, rows, key_columns, key_sizes, stored_block, null_map, pool); - }); - } - /// If RIGHT or FULL save blocks with nulls for NonJoinedBlockInputStream + UInt8 save_nullmap = 0; if (isRightOrFull(kind) && null_map) { - UInt8 has_null = 0; - for (size_t i = 0; !has_null && i < null_map->size(); ++i) - has_null |= (*null_map)[i]; - - if (has_null) - blocks_nullmaps.emplace_back(stored_block, null_map_holder); + for (size_t i = 0; !save_nullmap && i < null_map->size(); ++i) + save_nullmap |= (*null_map)[i]; } - return table_join->sizeLimits().check(getTotalRowCount(), getTotalByteCount(), "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); + Block structured_block = structureRightBlock(block); + size_t total_rows = 0; + size_t total_bytes = 0; + + { + std::unique_lock lock(rwlock); + + blocks.emplace_back(std::move(structured_block)); + Block * stored_block = &blocks.back(); + + size_t rows = block.rows(); + if (rows) + has_no_rows_in_maps = false; + + if (kind != ASTTableJoin::Kind::Cross) + { + joinDispatch(kind, strictness, maps, [&](auto, auto strictness_, auto & map) + { + insertFromBlockImpl(*this, type, map, rows, key_columns, key_sizes, stored_block, null_map, pool); + }); + } + + if (save_nullmap) + blocks_nullmaps.emplace_back(stored_block, null_map_holder); + + /// TODO: Do not calculate them every time + total_rows = getTotalRowCount(); + total_bytes = getTotalByteCount(); + } + + return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } From 3829df0e7b35bc0001ab30b6badd3dd0ddbf9dfa Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 3 Dec 2019 17:38:13 +0300 Subject: [PATCH 25/77] fix comment --- dbms/src/Interpreters/Join.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 6342bd891a8..ff46380db13 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -364,7 +364,7 @@ private: */ void setSampleBlock(const Block & block); - /// Modify (structure) and save right block, @returns pointer to saved block + /// Modify (structure) right block to save it in block list Block structureRightBlock(const Block & stored_block) const; void initRightBlockStructure(); void initRequiredRightKeys(); From 4f2f474fdef4a86ddf4c916dc0d7bc0d1cf2c06e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 Nov 2019 19:24:44 +0300 Subject: [PATCH 26/77] Add info about affected ports to IProcessor::prepare --- .../Processors/Executors/PipelineExecutor.cpp | 252 ++++++++++-------- .../Processors/Executors/PipelineExecutor.h | 27 +- dbms/src/Processors/IProcessor.h | 43 ++- dbms/src/Processors/ResizeProcessor.cpp | 107 ++++++++ dbms/src/Processors/ResizeProcessor.h | 37 +++ 5 files changed, 333 insertions(+), 133 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 8892418d0dc..3e2051df890 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -7,7 +7,6 @@ #include #include -#include #include #include #include @@ -52,26 +51,24 @@ bool PipelineExecutor::addEdges(UInt64 node) const IProcessor * cur = graph[node].processor; - auto add_edge = [&](auto & from_port, const IProcessor * to_proc, Edges & edges) + auto add_edge = [&](auto & from_port, const IProcessor * to_proc, Edges & edges, + bool is_backward, UInt64 input_port_number, UInt64 output_port_number) { auto it = processors_map.find(to_proc); if (it == processors_map.end()) throwUnknownProcessor(to_proc, cur, true); UInt64 proc_num = it->second; - Edge * edge_ptr = nullptr; for (auto & edge : edges) - if (edge.to == proc_num) - edge_ptr = &edge; - - if (!edge_ptr) { - edge_ptr = &edges.emplace_back(); - edge_ptr->to = proc_num; + if (edge.to == proc_num) + throw Exception("Multiple edges are not allowed for the same processors.", ErrorCodes::LOGICAL_ERROR); } - from_port.setVersion(&edge_ptr->version); + auto & edge = edges.emplace_back(proc_num, is_backward, input_port_number, output_port_number); + + from_port.setVersion(&edge.version); }; bool was_edge_added = false; @@ -83,10 +80,11 @@ bool PipelineExecutor::addEdges(UInt64 node) { was_edge_added = true; - for (auto it = std::next(inputs.begin(), from_input); it != inputs.end(); ++it) + for (auto it = std::next(inputs.begin(), from_input); it != inputs.end(); ++it, ++from_input) { const IProcessor * proc = &it->getOutputPort().getProcessor(); - add_edge(*it, proc, graph[node].backEdges); + auto output_port_number = proc->getOutputPortNumber(&it->getOutputPort()); + add_edge(*it, proc, graph[node].backEdges, true, from_input, output_port_number); } } @@ -97,10 +95,11 @@ bool PipelineExecutor::addEdges(UInt64 node) { was_edge_added = true; - for (auto it = std::next(outputs.begin(), from_output); it != outputs.end(); ++it) + for (auto it = std::next(outputs.begin(), from_output); it != outputs.end(); ++it, ++from_output) { const IProcessor * proc = &it->getInputPort().getProcessor(); - add_edge(*it, proc, graph[node].directEdges); + auto input_port_number = proc->getInputPortNumber(&it->getInputPort()); + add_edge(*it, proc, graph[node].directEdges, false, input_port_number, from_output); } } @@ -131,6 +130,7 @@ void PipelineExecutor::addChildlessProcessorsToStack(Stack & stack) if (graph[proc].directEdges.empty()) { stack.push(proc); + /// do not lock mutex, as this function is executedin single thread graph[proc].status = ExecStatus::Preparing; } } @@ -195,9 +195,20 @@ void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid) UInt64 num_processors = processors.size(); for (UInt64 node = 0; node < num_processors; ++node) { + size_t num_direct_edges = graph[node].directEdges.size(); + size_t num_back_edges = graph[node].backEdges.size(); + if (addEdges(node)) { - if (graph[node].status == ExecStatus::Idle || graph[node].status == ExecStatus::New) + std::lock_guard guard(graph[node].status_mutex); + + for (; num_back_edges < graph[node].backEdges.size(); ++num_back_edges) + graph[node].updated_input_ports.emplace_back(num_back_edges); + + for (; num_direct_edges < graph[node].directEdges.size(); ++num_direct_edges) + graph[node].updated_output_ports.emplace_back(num_direct_edges); + + if (graph[node].status == ExecStatus::Idle) { graph[node].status = ExecStatus::Preparing; stack.push(node); @@ -212,140 +223,147 @@ bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stac auto & node = graph[edge.to]; - ExecStatus status = node.status.load(); + std::lock_guard guard(node.status_mutex); - /// Don't add processor if nothing was read from port. - if (status != ExecStatus::New && edge.version == edge.prev_version) - return false; + ExecStatus status = node.status; if (status == ExecStatus::Finished) return false; - /// Signal that node need to be prepared. - node.need_to_be_prepared = true; - edge.prev_version = edge.version; + if (edge.backward) + node.updated_output_ports.push_back(edge.output_port_number); + else + node.updated_input_ports.push_back(edge.input_port_number); - /// Try to get ownership for node. - - /// Assume that current status is New or Idle. Otherwise, can't prepare node. - if (status != ExecStatus::New) - status = ExecStatus::Idle; - - /// Statuses but New and Idle are not interesting because they own node. - /// Prepare will be called in owning thread before changing status. - while (!node.status.compare_exchange_weak(status, ExecStatus::Preparing)) - if (!(status == ExecStatus::New || status == ExecStatus::Idle) || !node.need_to_be_prepared) - return false; - - stack.push(edge.to); - return true; + if (status == ExecStatus::Idle) + { + node.status = ExecStatus::Preparing; + stack.push(edge.to); + return true; + } + return false; } bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & parents, size_t thread_number, bool async) { + + /// In this method we have ownership on node. auto & node = graph[pid]; + bool need_traverse = false; + bool need_expand_pipeline = false; + + std::vector updated_back_edges; + std::vector updated_direct_edges; + { /// Stopwatch watch; - /// Disable flag before prepare call. Otherwise, we can skip prepare request. - /// Prepare can be called more times than needed, but it's ok. - node.need_to_be_prepared = false; + std::lock_guard guard(node.status_mutex); - auto status = node.processor->prepare(); + auto status = node.processor->prepare(node.updated_input_ports, node.updated_output_ports); + node.updated_input_ports.clear(); + node.updated_output_ports.clear(); /// node.execution_state->preparation_time_ns += watch.elapsed(); node.last_processor_status = status; - } - auto add_neighbours_to_prepare_queue = [&] () - { - for (auto & edge : node.backEdges) - tryAddProcessorToStackIfUpdated(edge, parents); - - for (auto & edge : node.directEdges) - tryAddProcessorToStackIfUpdated(edge, children); - }; - - auto try_release_ownership = [&] () - { - /// This function can be called after expand pipeline, where node from outer scope is not longer valid. - auto & node_ = graph[pid]; - ExecStatus expected = ExecStatus::Idle; - node_.status = ExecStatus::Idle; - - if (node_.need_to_be_prepared) + switch (node.last_processor_status) { - while (!node_.status.compare_exchange_weak(expected, ExecStatus::Preparing)) - if (!(expected == ExecStatus::Idle) || !node_.need_to_be_prepared) - return; - - children.push(pid); - } - }; - - switch (node.last_processor_status) - { - case IProcessor::Status::NeedData: - case IProcessor::Status::PortFull: - { - add_neighbours_to_prepare_queue(); - try_release_ownership(); - - break; - } - case IProcessor::Status::Finished: - { - add_neighbours_to_prepare_queue(); - node.status = ExecStatus::Finished; - break; - } - case IProcessor::Status::Ready: - { - node.status = ExecStatus::Executing; - return true; - } - case IProcessor::Status::Async: - { - throw Exception("Async is temporary not supported.", ErrorCodes::LOGICAL_ERROR); + case IProcessor::Status::NeedData: + case IProcessor::Status::PortFull: + { + need_traverse = true; + node.status = ExecStatus::Idle; + break; + } + case IProcessor::Status::Finished: + { + need_traverse = true; + node.status = ExecStatus::Finished; + break; + } + case IProcessor::Status::Ready: + { + node.status = ExecStatus::Executing; + return true; + } + case IProcessor::Status::Async: + { + throw Exception("Async is temporary not supported.", ErrorCodes::LOGICAL_ERROR); // node.status = ExecStatus::Executing; // addAsyncJob(pid); // break; - } - case IProcessor::Status::Wait: - { - if (!async) - throw Exception("Processor returned status Wait before Async.", ErrorCodes::LOGICAL_ERROR); - break; - } - case IProcessor::Status::ExpandPipeline: - { - executor_contexts[thread_number]->task_list.emplace_back( - node.execution_state.get(), - &parents - ); - - ExpandPipelineTask * desired = &executor_contexts[thread_number]->task_list.back(); - ExpandPipelineTask * expected = nullptr; - - while (!expand_pipeline_task.compare_exchange_strong(expected, desired)) + } + case IProcessor::Status::Wait: { - doExpandPipeline(expected, true); - expected = nullptr; + if (!async) + throw Exception("Processor returned status Wait before Async.", ErrorCodes::LOGICAL_ERROR); + break; + } + case IProcessor::Status::ExpandPipeline: + { + need_expand_pipeline = true; + break; + } + } + + if (need_traverse) + { + for (auto & edge : node.backEdges) + { + if (edge.version != edge.prev_version) + { + updated_back_edges.emplace_back(&edge); + edge.prev_version = edge.version; + } } - doExpandPipeline(desired, true); - - /// node is not longer valid after pipeline was expanded - graph[pid].need_to_be_prepared = true; - try_release_ownership(); - break; + for (auto & edge : node.directEdges) + { + if (edge.version != edge.prev_version) + { + updated_direct_edges.emplace_back(&edge); + edge.prev_version = edge.version; + } + } } } + if (need_traverse) + { + for (auto & edge : updated_back_edges) + tryAddProcessorToStackIfUpdated(*edge, parents); + + for (auto & edge : updated_direct_edges) + tryAddProcessorToStackIfUpdated(*edge, children); + } + + if (need_expand_pipeline) + { + executor_contexts[thread_number]->task_list.emplace_back( + node.execution_state.get(), + &parents + ); + + ExpandPipelineTask * desired = &executor_contexts[thread_number]->task_list.back(); + ExpandPipelineTask * expected = nullptr; + + while (!expand_pipeline_task.compare_exchange_strong(expected, desired)) + { + doExpandPipeline(expected, true); + expected = nullptr; + } + + doExpandPipeline(desired, true); + + /// Add itself back to be prepared again. + children.push(pid); + } + return false; } @@ -427,7 +445,7 @@ void PipelineExecutor::execute(size_t num_threads) bool all_processors_finished = true; for (auto & node : graph) - if (node.status != ExecStatus::Finished) + if (node.status != ExecStatus::Finished) /// Single thread, do not hold mutex all_processors_finished = false; if (!all_processors_finished) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index b5e3c7a0e1e..37a72c69991 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -43,12 +43,18 @@ private: struct Edge { + Edge(UInt64 to_, bool backward_, UInt64 input_port_number_, UInt64 output_port_number_) + : to(to_), backward(backward_), input_port_number(input_port_number_), output_port_number(output_port_number_) {} + UInt64 to = std::numeric_limits::max(); + bool backward; + UInt64 input_port_number; + UInt64 output_port_number; /// Edge version is increased when port's state is changed (e.g. when data is pushed). See Port.h for details. /// To compare version with prev_version we can decide if neighbour processor need to be prepared. - UInt64 version = 0; - UInt64 prev_version = 0; + UInt64 version = 1; + UInt64 prev_version = 0; /// prev version is zero so ve traverse all edges after the first prepare. }; /// Use std::list because new ports can be added to processor during execution. @@ -58,7 +64,6 @@ private: /// Can be owning or not. Owning means that executor who set this status can change node's data and nobody else can. enum class ExecStatus { - New, /// prepare wasn't called yet. Initial state. Non-owning. Idle, /// prepare returned NeedData or PortFull. Non-owning. Preparing, /// some executor is preparing processor, or processor is in task_queue. Owning. Executing, /// prepare returned Ready and task is executing. Owning. @@ -87,17 +92,19 @@ private: Edges directEdges; Edges backEdges; - std::atomic status; - /// This flag can be set by any executor. - /// When enabled, any executor can try to atomically set Preparing state to status. - std::atomic_bool need_to_be_prepared; + ExecStatus status; + std::mutex status_mutex; + /// Last state for profiling. IProcessor::Status last_processor_status = IProcessor::Status::NeedData; std::unique_ptr execution_state; + IProcessor::PortNumbers updated_input_ports; + IProcessor::PortNumbers updated_output_ports; + Node(IProcessor * processor_, UInt64 processor_id) - : processor(processor_), status(ExecStatus::New), need_to_be_prepared(false) + : processor(processor_), status(ExecStatus::Idle) { execution_state = std::make_unique(); execution_state->processor = processor; @@ -105,8 +112,8 @@ private: } Node(Node && other) noexcept - : processor(other.processor), status(other.status.load()) - , need_to_be_prepared(other.need_to_be_prepared.load()), execution_state(std::move(other.execution_state)) + : processor(other.processor), status(other.status) + , execution_state(std::move(other.execution_state)) { } }; diff --git a/dbms/src/Processors/IProcessor.h b/dbms/src/Processors/IProcessor.h index ed59f4e591d..852bde2d467 100644 --- a/dbms/src/Processors/IProcessor.h +++ b/dbms/src/Processors/IProcessor.h @@ -171,7 +171,15 @@ public: * - method 'prepare' cannot be executed in parallel even for different objects, * if they are connected (including indirectly) to each other by their ports; */ - virtual Status prepare() = 0; + virtual Status prepare() + { + throw Exception("Method 'prepare' is not implemented for " + getName() + " processor", ErrorCodes::NOT_IMPLEMENTED); + } + + using PortNumbers = std::vector; + + /// Optimization for prepare in case we know ports were updated. + virtual Status prepare(const PortNumbers & /*updated_input_ports*/, const PortNumbers & /*updated_output_ports*/) { return prepare(); } /** You may call this method if 'prepare' returned Ready. * This method cannot access any ports. It should use only data that was prepared by 'prepare' method. @@ -183,11 +191,6 @@ public: throw Exception("Method 'work' is not implemented for " + getName() + " processor", ErrorCodes::NOT_IMPLEMENTED); } - virtual void work(size_t /*thread_num*/) - { - work(); - } - /** You may call this method if 'prepare' returned Async. * This method cannot access any ports. It should use only data that was prepared by 'prepare' method. * @@ -226,6 +229,34 @@ public: auto & getInputs() { return inputs; } auto & getOutputs() { return outputs; } + UInt64 getInputPortNumber(const InputPort * input_port) const + { + UInt64 number = 0; + for (auto & port : inputs) + { + if (&port == input_port) + return number; + + ++number; + } + + throw Exception("Can't find input port for " + getName() + " processor", ErrorCodes::LOGICAL_ERROR); + } + + UInt64 getOutputPortNumber(const OutputPort * output_port) const + { + UInt64 number = 0; + for (auto & port : outputs) + { + if (&port == output_port) + return number; + + ++number; + } + + throw Exception("Can't find output port for " + getName() + " processor", ErrorCodes::LOGICAL_ERROR); + } + const auto & getInputs() const { return inputs; } const auto & getOutputs() const { return outputs; } diff --git a/dbms/src/Processors/ResizeProcessor.cpp b/dbms/src/Processors/ResizeProcessor.cpp index b3cb3a1735d..2ba1dd56275 100644 --- a/dbms/src/Processors/ResizeProcessor.cpp +++ b/dbms/src/Processors/ResizeProcessor.cpp @@ -153,5 +153,112 @@ ResizeProcessor::Status ResizeProcessor::prepare() return get_status_if_no_inputs(); } +IProcessor::Status ResizeProcessor::prepare(const PortNumbers & updated_inputs, const PortNumbers & updated_outputs) +{ + if (!initialized) + { + initialized = true; + + for (auto & input : inputs) + { + input.setNeeded(); + input_ports.push_back({.port = &input, .status = InputStatus::NotActive}); + } + + for (auto & output : outputs) + output_ports.push_back({.port = &output, .status = OutputStatus::NotActive}); + } + + for (auto & output_number : updated_outputs) + { + auto & output = output_ports[output_number]; + if (output.port->isFinished()) + { + if (output.status != OutputStatus::Finished) + { + ++num_finished_outputs; + output.status = OutputStatus::Finished; + } + + continue; + } + + if (output.port->canPush()) + { + if (output.status != OutputStatus::NeedData) + { + output.status = OutputStatus::NeedData; + waiting_outputs.push(output_number); + } + } + } + + if (num_finished_outputs == outputs.size()) + { + for (auto & input : inputs) + input.close(); + + return Status::Finished; + } + + for (auto & input_number : updated_inputs) + { + auto & input = input_ports[input_number]; + if (input.port->isFinished()) + { + if (input.status != InputStatus::Finished) + { + input.status = InputStatus::Finished; + ++num_finished_inputs; + } + continue; + } + + if (input.port->hasData()) + { + if (input.status != InputStatus::HasData) + { + input.status = InputStatus::HasData; + inputs_with_data.push(input_number); + } + } + } + + while (!waiting_outputs.empty() && !inputs_with_data.empty()) + { + auto & waiting_output = output_ports[waiting_outputs.front()]; + waiting_outputs.pop(); + + auto & input_with_data = input_ports[inputs_with_data.front()]; + inputs_with_data.pop(); + + waiting_output.port->pushData(input_with_data.port->pullData()); + input_with_data.status = InputStatus::NotActive; + waiting_output.status = OutputStatus::NotActive; + + if (input_with_data.port->isFinished()) + { + if (input_with_data.status != InputStatus::Finished) + { + input_with_data.status = InputStatus::Finished; + ++num_finished_inputs; + } + } + } + + if (num_finished_inputs == inputs.size()) + { + for (auto & output : outputs) + output.finish(); + + return Status::Finished; + } + + if (!waiting_outputs.empty()) + return Status::NeedData; + + return Status::PortFull; +} + } diff --git a/dbms/src/Processors/ResizeProcessor.h b/dbms/src/Processors/ResizeProcessor.h index 67574c384a1..3a9c906ecbd 100644 --- a/dbms/src/Processors/ResizeProcessor.h +++ b/dbms/src/Processors/ResizeProcessor.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -31,10 +32,46 @@ public: String getName() const override { return "Resize"; } Status prepare() override; + Status prepare(const PortNumbers &, const PortNumbers &) override; private: InputPorts::iterator current_input; OutputPorts::iterator current_output; + + size_t num_finished_inputs = 0; + size_t num_finished_outputs = 0; + std::queue waiting_outputs; + std::queue inputs_with_data; + bool initialized = false; + + enum class OutputStatus + { + NotActive, + NeedData, + Finished, + }; + + enum class InputStatus + { + NotActive, + HasData, + Finished, + }; + + struct InputPortWithStatus + { + InputPort * port; + InputStatus status; + }; + + struct OutputPortWithStatus + { + OutputPort * port; + OutputStatus status; + }; + + std::vector input_ports; + std::vector output_ports; }; } From 9465183408253661c75025d08f2045ff602f214d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 2 Dec 2019 20:32:16 +0300 Subject: [PATCH 27/77] Add info about affected ports after prepare. --- .../Processors/Executors/PipelineExecutor.cpp | 34 +++++------ .../Processors/Executors/PipelineExecutor.h | 16 +++-- dbms/src/Processors/Port.h | 59 ++++++++++++------- 3 files changed, 68 insertions(+), 41 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 3e2051df890..8bf2a9e2948 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -52,7 +52,8 @@ bool PipelineExecutor::addEdges(UInt64 node) const IProcessor * cur = graph[node].processor; auto add_edge = [&](auto & from_port, const IProcessor * to_proc, Edges & edges, - bool is_backward, UInt64 input_port_number, UInt64 output_port_number) + bool is_backward, UInt64 input_port_number, UInt64 output_port_number, + std::vector * update_list) { auto it = processors_map.find(to_proc); if (it == processors_map.end()) @@ -66,9 +67,9 @@ bool PipelineExecutor::addEdges(UInt64 node) throw Exception("Multiple edges are not allowed for the same processors.", ErrorCodes::LOGICAL_ERROR); } - auto & edge = edges.emplace_back(proc_num, is_backward, input_port_number, output_port_number); + auto & edge = edges.emplace_back(proc_num, is_backward, input_port_number, output_port_number, update_list); - from_port.setVersion(&edge.version); + from_port.setUpdateInfo(&edge.update_info); }; bool was_edge_added = false; @@ -84,7 +85,7 @@ bool PipelineExecutor::addEdges(UInt64 node) { const IProcessor * proc = &it->getOutputPort().getProcessor(); auto output_port_number = proc->getOutputPortNumber(&it->getOutputPort()); - add_edge(*it, proc, graph[node].backEdges, true, from_input, output_port_number); + add_edge(*it, proc, graph[node].backEdges, true, from_input, output_port_number, &graph[node].post_updated_input_ports); } } @@ -99,7 +100,7 @@ bool PipelineExecutor::addEdges(UInt64 node) { const IProcessor * proc = &it->getInputPort().getProcessor(); auto input_port_number = proc->getInputPortNumber(&it->getInputPort()); - add_edge(*it, proc, graph[node].directEdges, false, input_port_number, from_output); + add_edge(*it, proc, graph[node].directEdges, false, input_port_number, from_output, &graph[node].post_updated_output_ports); } } @@ -313,23 +314,22 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa if (need_traverse) { - for (auto & edge : node.backEdges) + for (auto & edge_id : node.post_updated_input_ports) { - if (edge.version != edge.prev_version) - { - updated_back_edges.emplace_back(&edge); - edge.prev_version = edge.version; - } + auto edge = static_cast(edge_id); + updated_back_edges.emplace_back(edge); + edge->update_info.trigger(); } - for (auto & edge : node.directEdges) + for (auto & edge_id : node.post_updated_output_ports) { - if (edge.version != edge.prev_version) - { - updated_direct_edges.emplace_back(&edge); - edge.prev_version = edge.version; - } + auto edge = static_cast(edge_id); + updated_direct_edges.emplace_back(edge); + edge->update_info.trigger(); } + + node.post_updated_input_ports.clear(); + node.post_updated_output_ports.clear(); } } diff --git a/dbms/src/Processors/Executors/PipelineExecutor.h b/dbms/src/Processors/Executors/PipelineExecutor.h index 37a72c69991..aded3de3008 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.h +++ b/dbms/src/Processors/Executors/PipelineExecutor.h @@ -43,8 +43,14 @@ private: struct Edge { - Edge(UInt64 to_, bool backward_, UInt64 input_port_number_, UInt64 output_port_number_) - : to(to_), backward(backward_), input_port_number(input_port_number_), output_port_number(output_port_number_) {} + Edge(UInt64 to_, bool backward_, + UInt64 input_port_number_, UInt64 output_port_number_, std::vector * update_list) + : to(to_), backward(backward_) + , input_port_number(input_port_number_), output_port_number(output_port_number_) + { + update_info.update_list = update_list; + update_info.id = this; + } UInt64 to = std::numeric_limits::max(); bool backward; @@ -53,8 +59,7 @@ private: /// Edge version is increased when port's state is changed (e.g. when data is pushed). See Port.h for details. /// To compare version with prev_version we can decide if neighbour processor need to be prepared. - UInt64 version = 1; - UInt64 prev_version = 0; /// prev version is zero so ve traverse all edges after the first prepare. + Port::UpdateInfo update_info; }; /// Use std::list because new ports can be added to processor during execution. @@ -95,6 +100,9 @@ private: ExecStatus status; std::mutex status_mutex; + std::vector post_updated_input_ports; + std::vector post_updated_output_ports; + /// Last state for profiling. IProcessor::Status last_processor_status = IProcessor::Status::NeedData; diff --git a/dbms/src/Processors/Port.h b/dbms/src/Processors/Port.h index d37e11668e7..54002669787 100644 --- a/dbms/src/Processors/Port.h +++ b/dbms/src/Processors/Port.h @@ -28,6 +28,25 @@ class Port friend void connect(OutputPort &, InputPort &); friend class IProcessor; +public: + struct UpdateInfo + { + std::vector * update_list = nullptr; + void * id = nullptr; + UInt64 version = 0; + UInt64 prev_version = 0; + + void inline ALWAYS_INLINE update() + { + if (version == prev_version && update_list) + update_list->push_back(id); + + ++version; + } + + void inline ALWAYS_INLINE trigger() { prev_version = version; } + }; + protected: /// Shared state of two connected ports. class State @@ -182,12 +201,17 @@ protected: IProcessor * processor = nullptr; + /// If update_info was set, will call update() for it in case port's state have changed. + UpdateInfo * update_info; + public: using Data = State::Data; Port(Block header_) : header(std::move(header_)) {} Port(Block header_, IProcessor * processor_) : header(std::move(header_)), processor(processor_) {} + void setUpdateInfo(UpdateInfo * info) { update_info = info; } + const Block & getHeader() const { return header; } bool ALWAYS_INLINE isConnected() const { return state != nullptr; } @@ -216,6 +240,13 @@ public: throw Exception("Port does not belong to Processor", ErrorCodes::LOGICAL_ERROR); return *processor; } + +protected: + void inline ALWAYS_INLINE updateVersion() + { + if (likely(update_info)) + update_info->update(); + } }; /// Invariants: @@ -230,20 +261,14 @@ class InputPort : public Port private: OutputPort * output_port = nullptr; - /// If version was set, it will be increased on each pull. - UInt64 * version = nullptr; - mutable bool is_finished = false; public: using Port::Port; - void setVersion(UInt64 * value) { version = value; } - Data ALWAYS_INLINE pullData() { - if (version) - ++(*version); + updateVersion(); assumeConnected(); @@ -296,8 +321,8 @@ public: { assumeConnected(); - if ((state->setFlags(State::IS_NEEDED, State::IS_NEEDED) & State::IS_NEEDED) == 0 && version) - ++(*version); + if ((state->setFlags(State::IS_NEEDED, State::IS_NEEDED) & State::IS_NEEDED) == 0) + updateVersion(); } void ALWAYS_INLINE setNotNeeded() @@ -310,8 +335,8 @@ public: { assumeConnected(); - if ((state->setFlags(State::IS_FINISHED, State::IS_FINISHED) & State::IS_FINISHED) == 0 && version) - ++(*version); + if ((state->setFlags(State::IS_FINISHED, State::IS_FINISHED) & State::IS_FINISHED) == 0) + updateVersion(); is_finished = true; } @@ -342,14 +367,9 @@ class OutputPort : public Port private: InputPort * input_port = nullptr; - /// If version was set, it will be increased on each push. - UInt64 * version = nullptr; - public: using Port::Port; - void setVersion(UInt64 * value) { version = value; } - void ALWAYS_INLINE push(Chunk chunk) { pushData({.chunk = std::move(chunk), .exception = {}}); @@ -374,8 +394,7 @@ public: throw Exception(msg, ErrorCodes::LOGICAL_ERROR); } - if (version) - ++(*version); + updateVersion(); assumeConnected(); @@ -390,8 +409,8 @@ public: auto flags = state->setFlags(State::IS_FINISHED, State::IS_FINISHED); - if (version && (flags & State::IS_FINISHED) == 0) - ++(*version); + if ((flags & State::IS_FINISHED) == 0) + updateVersion(); } bool ALWAYS_INLINE isNeeded() const From 8524a7558054b183f55139cec1415cdc76db3eab Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 2 Dec 2019 21:33:25 +0300 Subject: [PATCH 28/77] Check once if filter column can be always false in filter transform. --- .../src/Processors/Transforms/FilterTransform.cpp | 15 +++++++++++++-- dbms/src/Processors/Transforms/FilterTransform.h | 2 ++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Transforms/FilterTransform.cpp b/dbms/src/Processors/Transforms/FilterTransform.cpp index 058df590f0c..3defbff1582 100644 --- a/dbms/src/Processors/Transforms/FilterTransform.cpp +++ b/dbms/src/Processors/Transforms/FilterTransform.cpp @@ -64,8 +64,7 @@ FilterTransform::FilterTransform( IProcessor::Status FilterTransform::prepare() { - if (constant_filter_description.always_false - || expression->checkColumnIsAlwaysFalse(filter_column_name)) + if (constant_filter_description.always_false) { input.close(); output.finish(); @@ -84,6 +83,18 @@ void FilterTransform::removeFilterIfNeed(Chunk & chunk) void FilterTransform::transform(Chunk & chunk) { + if (!initialized) + { + initialized = true; + + if (expression->checkColumnIsAlwaysFalse(filter_column_name)) + { + stopReading(); + chunk = Chunk(getOutputPort().getHeader().getColumns(), 0); + return; + } + } + size_t num_rows_before_filtration = chunk.getNumRows(); auto columns = chunk.detachColumns(); diff --git a/dbms/src/Processors/Transforms/FilterTransform.h b/dbms/src/Processors/Transforms/FilterTransform.h index 127eb5a8039..c595d72a70b 100644 --- a/dbms/src/Processors/Transforms/FilterTransform.h +++ b/dbms/src/Processors/Transforms/FilterTransform.h @@ -36,6 +36,8 @@ private: /// Header after expression, but before removing filter column. Block transformed_header; + bool initialized = false; + void removeFilterIfNeed(Chunk & chunk); }; From 4337fc20b537c2072182eefa1d5290ae41ce3f41 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 2 Dec 2019 21:34:46 +0300 Subject: [PATCH 29/77] Enable processors by default. --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 06b77e22ca5..138f08f6bb5 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -366,7 +366,7 @@ struct Settings : public SettingsCollection M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.", 0) \ \ - M(SettingBool, experimental_use_processors, false, "Use processors pipeline.", 0) \ + M(SettingBool, experimental_use_processors, true, "Use processors pipeline.", 0) \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ From fa98587c192d812336f650136d7bb67485cd2969 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Dec 2019 11:50:19 +0300 Subject: [PATCH 30/77] Init update_info in port. --- dbms/src/Processors/Port.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Processors/Port.h b/dbms/src/Processors/Port.h index 54002669787..9ea3f3f4190 100644 --- a/dbms/src/Processors/Port.h +++ b/dbms/src/Processors/Port.h @@ -202,7 +202,7 @@ protected: IProcessor * processor = nullptr; /// If update_info was set, will call update() for it in case port's state have changed. - UpdateInfo * update_info; + UpdateInfo * update_info = nullptr; public: using Data = State::Data; From 81b5b17828c1127466576dc1319cc1cd60d22cae Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Dec 2019 15:08:59 +0300 Subject: [PATCH 31/77] Fix pvs studio varning. --- dbms/src/Processors/ResizeProcessor.cpp | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/dbms/src/Processors/ResizeProcessor.cpp b/dbms/src/Processors/ResizeProcessor.cpp index 2ba1dd56275..59d1f0db75e 100644 --- a/dbms/src/Processors/ResizeProcessor.cpp +++ b/dbms/src/Processors/ResizeProcessor.cpp @@ -238,11 +238,8 @@ IProcessor::Status ResizeProcessor::prepare(const PortNumbers & updated_inputs, if (input_with_data.port->isFinished()) { - if (input_with_data.status != InputStatus::Finished) - { - input_with_data.status = InputStatus::Finished; - ++num_finished_inputs; - } + input_with_data.status = InputStatus::Finished; + ++num_finished_inputs; } } From efe1616fdabe2207744f6ec0bb7cefce0cbefd5a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 3 Dec 2019 15:27:44 +0300 Subject: [PATCH 32/77] Fix optimization for 'WHERE column in (empty set)' case. --- .../Processors/Transforms/FilterTransform.cpp | 26 +++++++++---------- .../Processors/Transforms/FilterTransform.h | 2 +- 2 files changed, 13 insertions(+), 15 deletions(-) diff --git a/dbms/src/Processors/Transforms/FilterTransform.cpp b/dbms/src/Processors/Transforms/FilterTransform.cpp index 3defbff1582..9cad9f85f92 100644 --- a/dbms/src/Processors/Transforms/FilterTransform.cpp +++ b/dbms/src/Processors/Transforms/FilterTransform.cpp @@ -64,14 +64,24 @@ FilterTransform::FilterTransform( IProcessor::Status FilterTransform::prepare() { - if (constant_filter_description.always_false) + if (constant_filter_description.always_false + /// Optimization for `WHERE column in (empty set)`. + /// The result will not change after set was created, so we can skip this check. + /// It is implemented in prepare() stop pipeline before reading from input port. + || (!are_prepared_sets_initialized && expression->checkColumnIsAlwaysFalse(filter_column_name))) { input.close(); output.finish(); return Status::Finished; } - return ISimpleTransform::prepare(); + auto status = ISimpleTransform::prepare(); + + /// Until prepared sets are initialized, output port will be unneeded, and prepare will return PortFull. + if (status != IProcessor::Status::PortFull) + are_prepared_sets_initialized = true; + + return status; } @@ -83,18 +93,6 @@ void FilterTransform::removeFilterIfNeed(Chunk & chunk) void FilterTransform::transform(Chunk & chunk) { - if (!initialized) - { - initialized = true; - - if (expression->checkColumnIsAlwaysFalse(filter_column_name)) - { - stopReading(); - chunk = Chunk(getOutputPort().getHeader().getColumns(), 0); - return; - } - } - size_t num_rows_before_filtration = chunk.getNumRows(); auto columns = chunk.detachColumns(); diff --git a/dbms/src/Processors/Transforms/FilterTransform.h b/dbms/src/Processors/Transforms/FilterTransform.h index c595d72a70b..1652473aa3c 100644 --- a/dbms/src/Processors/Transforms/FilterTransform.h +++ b/dbms/src/Processors/Transforms/FilterTransform.h @@ -36,7 +36,7 @@ private: /// Header after expression, but before removing filter column. Block transformed_header; - bool initialized = false; + bool are_prepared_sets_initialized = false; void removeFilterIfNeed(Chunk & chunk); }; From 901a6b4627c5392974a7a3c573d43978cee2964d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 4 Dec 2019 12:56:08 +0300 Subject: [PATCH 33/77] Style fix. --- dbms/src/Processors/Executors/PipelineExecutor.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 8bf2a9e2948..9013b83486a 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -248,8 +248,6 @@ bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stac bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & parents, size_t thread_number, bool async) { - - /// In this method we have ownership on node. auto & node = graph[pid]; From 516876becbdb99db1ee89804e47b9c751bdf5364 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 15:39:09 +0300 Subject: [PATCH 34/77] Fix unintendent dependency from GLIBC 2.16 in clickhouse-odbc-bridge --- dbms/CMakeLists.txt | 2 ++ dbms/programs/odbc-bridge/CMakeLists.txt | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index fecc1fa7e76..eb4dd9550b4 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -432,6 +432,8 @@ if (USE_JEMALLOC) if(NOT MAKE_STATIC_LIBRARIES AND ${JEMALLOC_LIBRARIES} MATCHES "${CMAKE_STATIC_LIBRARY_SUFFIX}$") # mallctl in dbms/src/Interpreters/AsynchronousMetrics.cpp + # Actually we link JEMALLOC to almost all libraries. + # This is just hotfix for some uninvestigated problem. target_link_libraries(clickhouse_interpreters PRIVATE ${JEMALLOC_LIBRARIES}) endif() endif () diff --git a/dbms/programs/odbc-bridge/CMakeLists.txt b/dbms/programs/odbc-bridge/CMakeLists.txt index d03ff257562..460dfd007d4 100644 --- a/dbms/programs/odbc-bridge/CMakeLists.txt +++ b/dbms/programs/odbc-bridge/CMakeLists.txt @@ -30,6 +30,11 @@ if (Poco_Data_FOUND) set(CLICKHOUSE_ODBC_BRIDGE_LINK ${CLICKHOUSE_ODBC_BRIDGE_LINK} PRIVATE ${Poco_Data_LIBRARY}) set(CLICKHOUSE_ODBC_BRIDGE_INCLUDE ${CLICKHOUSE_ODBC_BRIDGE_INCLUDE} SYSTEM PRIVATE ${Poco_Data_INCLUDE_DIR}) endif () +if (USE_JEMALLOC) + # We need to link jemalloc directly to odbc-bridge-library, because in other case + # we will build it with default malloc. + set(CLICKHOUSE_ODBC_BRIDGE_LINK ${CLICKHOUSE_ODBC_BRIDGE_LINK} PRIVATE ${JEMALLOC_LIBRARIES}) +endif() clickhouse_program_add_library(odbc-bridge) From 3ac103f21cdc333a18c84c26552957fc3e655f3a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 4 Dec 2019 16:45:42 +0300 Subject: [PATCH 35/77] Fix the number of rows in MergingSortedTransform. --- dbms/src/Processors/Transforms/MergingSortedTransform.h | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.h b/dbms/src/Processors/Transforms/MergingSortedTransform.h index 0991835bfaf..b32dd076c5f 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.h +++ b/dbms/src/Processors/Transforms/MergingSortedTransform.h @@ -59,8 +59,11 @@ protected: auto num_rows = chunk.getNumRows(); columns = chunk.mutateColumns(); if (limit_rows && num_rows > limit_rows) + { + num_rows = limit_rows; for (auto & column : columns) - column = (*column->cut(0, limit_rows)->convertToFullColumnIfConst()).mutate(); + column = (*column->cut(0, num_rows)->convertToFullColumnIfConst()).mutate(); + } total_merged_rows += num_rows; merged_rows = num_rows; From 711fb5e66705324c1f7bac9d3fb9515511799ce7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 16:53:05 +0300 Subject: [PATCH 36/77] Added tests for external dictionary invalidate query failover --- ...ionary_invalidate_query_failover.reference | 5 ++ ...40_dictionary_invalidate_query_failover.sh | 80 +++++++++++++++++++ 2 files changed, 85 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.reference create mode 100755 dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh diff --git a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.reference b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.reference new file mode 100644 index 00000000000..1fca8dab675 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.reference @@ -0,0 +1,5 @@ +122 + +Table dictdb.dict_invalidate doesn\'t exist. + +133 diff --git a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh new file mode 100755 index 00000000000..5bd3bed581a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh @@ -0,0 +1,80 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + + +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb" + +$CLICKHOUSE_CLIENT --query "CREATE DATABASE dictdb Engine = Ordinary" + +$CLICKHOUSE_CLIENT --query " +CREATE TABLE dictdb.dict_invalidate +ENGINE = Memory AS +SELECT + 122 as dummy, + toDateTime('2019-10-29 18:51:35') AS last_time +FROM system.one" + + +$CLICKHOUSE_CLIENT --query " +CREATE DICTIONARY dictdb.invalidate +( + dummy UInt64, + two UInt8 EXPRESSION dummy +) +PRIMARY KEY dummy +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'dict_invalidate' DB 'dictdb' INVALIDATE_QUERY 'select max(last_time) from dictdb.dict_invalidate')) +LIFETIME(MIN 0 MAX 1) +LAYOUT(FLAT())" + +$CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUInt64(122))" + +$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" + +$CLICKHOUSE_CLIENT --query "DROP TABLE dictdb.dict_invalidate" + +function check_exception_detected() +{ + + query_result=`$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1` + + while [ -z "$query_result" ] + do + query_result=`$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1` + sleep 0.1 + done +} + + +export -f check_exception_detected; +timeout 10 bash -c check_exception_detected 2> /dev/null + +$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 | grep -Eo "Table dictdb.dict_invalidate .* exist." + +$CLICKHOUSE_CLIENT --query " +CREATE TABLE dictdb.dict_invalidate +ENGINE = Memory AS +SELECT + 133 as dummy, + toDateTime('2019-10-29 18:51:35') AS last_time +FROM system.one" + +function check_exception_fixed() +{ + query_result=`$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1` + + while [ "$query_result" ] + do + query_result=`$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1` + sleep 0.1 + done +} + +export -f check_exception_fixed; +timeout 10 bash -c check_exception_fixed 2> /dev/null + +$CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" 2>&1 +$CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUInt64(133))" + +$CLICKHOUSE_CLIENT --query "DROP DATABASE IF EXISTS dictdb" From ce05cea19262bcb8e82d9833846c64a90a0998f5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 17:22:49 +0300 Subject: [PATCH 37/77] Add sleep for test --- .../0_stateless/01040_dictionary_invalidate_query_failover.sh | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh index 5bd3bed581a..cbf7b352a86 100755 --- a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh +++ b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh @@ -32,6 +32,8 @@ $CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUI $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" +sleep 5 + $CLICKHOUSE_CLIENT --query "DROP TABLE dictdb.dict_invalidate" function check_exception_detected() From a41764cccc509ca07b4ba982c421f40b9a97ccd7 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 17:45:56 +0300 Subject: [PATCH 38/77] Reload dictionary with invalidate query if it has exception --- dbms/src/Interpreters/ExternalLoader.cpp | 7 +++++-- .../01040_dictionary_invalidate_query_failover.sh | 2 ++ 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 7bc5aedae6d..acf7632737c 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -603,14 +603,16 @@ public: continue; /// Object has been just loaded (it wasn't loaded while we were building the map `is_modified_map`), so we don't have to reload it right now. bool is_modified_flag = it->second; - if (!is_modified_flag) + /// Object maybe successfully loaded in some old state, but have an exception from new loads. + /// so even if it's not modified better to reload it. + if (!is_modified_flag && !info.hasException()) { /// Object wasn't modified so we only have to set `next_update_time`. info.next_update_time = calculateNextUpdateTime(info.object, info.error_count); continue; } - /// Object was modified and should be reloaded. + /// Object was modified or it's loaded (possible outdated state) with exception, so it should be reloaded. startLoading(name, info); } else if (info.failed()) @@ -633,6 +635,7 @@ private: bool loading() const { return loading_id != 0; } bool wasLoading() const { return loaded() || failed() || loading(); } bool ready() const { return (loaded() || failed()) && !forced_to_reload; } + bool hasException() { return exception != nullptr; } Status status() const { diff --git a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh index cbf7b352a86..ef5d3053f9a 100755 --- a/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh +++ b/dbms/tests/queries/0_stateless/01040_dictionary_invalidate_query_failover.sh @@ -32,6 +32,8 @@ $CLICKHOUSE_CLIENT --query "SELECT dictGetUInt8('dictdb.invalidate', 'two', toUI $CLICKHOUSE_CLIENT --query "SELECT last_exception FROM system.dictionaries WHERE database = 'dictdb' AND name = 'invalidate'" +# Bad solution, but it's quite complicated to detect, that invalidte_query stopped updates. +# In worst case we don't check anything, but fortunately it doesn't lead to false negatives. sleep 5 $CLICKHOUSE_CLIENT --query "DROP TABLE dictdb.dict_invalidate" From 3128b62f0e972a284950c35e8bf75bae0d9d40bb Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 18:11:36 +0300 Subject: [PATCH 39/77] Create dictionary if not exists --- dbms/src/Parsers/ParserCreateQuery.cpp | 6 +-- ..._create_dictionary_if_not_exists.reference | 2 + .../01041_create_dictionary_if_not_exists.sql | 40 +++++++++++++++++++ 3 files changed, 45 insertions(+), 3 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.reference create mode 100644 dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql diff --git a/dbms/src/Parsers/ParserCreateQuery.cpp b/dbms/src/Parsers/ParserCreateQuery.cpp index b0ca9c399ec..a014b861e77 100644 --- a/dbms/src/Parsers/ParserCreateQuery.cpp +++ b/dbms/src/Parsers/ParserCreateQuery.cpp @@ -849,12 +849,12 @@ bool ParserCreateDictionaryQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, E return false; } - if (s_if_not_exists.ignore(pos, expected)) - if_not_exists = true; - if (!s_dictionary.ignore(pos, expected)) return false; + if (s_if_not_exists.ignore(pos, expected)) + if_not_exists = true; + if (!name_p.parse(pos, name, expected)) return false; diff --git a/dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.reference b/dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.reference new file mode 100644 index 00000000000..15eecd22cf1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.reference @@ -0,0 +1,2 @@ +1.1 +1.1 diff --git a/dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql b/dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql new file mode 100644 index 00000000000..5002b7a59ab --- /dev/null +++ b/dbms/tests/queries/0_stateless/01041_create_dictionary_if_not_exists.sql @@ -0,0 +1,40 @@ +DROP DATABASE IF EXISTS dictdb; + +CREATE DATABASE dictdb ENGINE = Ordinary; + +CREATE TABLE dictdb.table_for_dict +( + key_column UInt64, + value Float64 +) +ENGINE = MergeTree() +ORDER BY key_column; + +INSERT INTO dictdb.table_for_dict VALUES (1, 1.1); + +CREATE DICTIONARY IF NOT EXISTS dictdb.dict_exists +( + key_column UInt64, + value Float64 DEFAULT 77.77 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'dictdb')) +LIFETIME(1) +LAYOUT(FLAT()); + +SELECT dictGetFloat64('dictdb.dict_exists', 'value', toUInt64(1)); + + +CREATE DICTIONARY IF NOT EXISTS dictdb.dict_exists +( + key_column UInt64, + value Float64 DEFAULT 77.77 +) +PRIMARY KEY key_column +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'table_for_dict' DB 'dictdb')) +LIFETIME(1) +LAYOUT(FLAT()); + +SELECT dictGetFloat64('dictdb.dict_exists', 'value', toUInt64(1)); + +DROP DATABASE IF EXISTS dictdb; From 40c69662238b3d3b5e3f7734113e639e31704737 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 18:23:05 +0300 Subject: [PATCH 40/77] Don't check dictionary modification if it's already have an exception. --- dbms/src/Interpreters/ExternalLoader.cpp | 31 ++++++++++++------------ 1 file changed, 16 insertions(+), 15 deletions(-) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index acf7632737c..2e7785700e2 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -560,8 +560,8 @@ public: /// The function doesn't touch the objects which were never tried to load. void reloadOutdated() { - /// Iterate through all the objects and find loaded ones which should be checked if they were modified. - std::unordered_map is_modified_map; + /// Iterate through all the objects and find loaded ones which should be checked if they need update. + std::unordered_map should_update_map; { std::lock_guard lock{mutex}; TimePoint now = std::chrono::system_clock::now(); @@ -569,22 +569,26 @@ public: { const auto & info = name_and_info.second; if ((now >= info.next_update_time) && !info.loading() && info.loaded()) - is_modified_map.emplace(info.object, true); + should_update_map.emplace(info.object, info.hasException()); } } /// Find out which of the loaded objects were modified. - /// We couldn't perform these checks while we were building `is_modified_map` because + /// We couldn't perform these checks while we were building `should_update_map` because /// the `mutex` should be unlocked while we're calling the function object->isModified() - for (auto & [object, is_modified_flag] : is_modified_map) + for (auto & [object, should_update_flag] : should_update_map) { try { - is_modified_flag = object->isModified(); + /// Maybe alredy true, if we have an exception + if (!should_update_flag) + should_update_flag = object->isModified(); } catch (...) { tryLogCurrentException(log, "Could not check if " + type_name + " '" + object->getName() + "' was modified"); + /// Cannot check isModified, so update + should_update_flag = true; } } @@ -598,16 +602,13 @@ public: { if (info.loaded()) { - auto it = is_modified_map.find(info.object); - if (it == is_modified_map.end()) - continue; /// Object has been just loaded (it wasn't loaded while we were building the map `is_modified_map`), so we don't have to reload it right now. + auto it = should_update_map.find(info.object); + if (it == should_update_map.end()) + continue; /// Object has been just loaded (it wasn't loaded while we were building the map `should_update_map`), so we don't have to reload it right now. - bool is_modified_flag = it->second; - /// Object maybe successfully loaded in some old state, but have an exception from new loads. - /// so even if it's not modified better to reload it. - if (!is_modified_flag && !info.hasException()) + bool should_update_flag = it->second; + if (!should_update_flag) { - /// Object wasn't modified so we only have to set `next_update_time`. info.next_update_time = calculateNextUpdateTime(info.object, info.error_count); continue; } @@ -635,7 +636,7 @@ private: bool loading() const { return loading_id != 0; } bool wasLoading() const { return loaded() || failed() || loading(); } bool ready() const { return (loaded() || failed()) && !forced_to_reload; } - bool hasException() { return exception != nullptr; } + bool hasException() const { return exception != nullptr; } Status status() const { From 7b8443710de33f2ffce0fd41c2a7e1a8c74c3156 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 19:20:24 +0300 Subject: [PATCH 41/77] Better code --- dbms/src/Interpreters/ExternalLoader.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 2e7785700e2..10b8a02d660 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -569,7 +569,7 @@ public: { const auto & info = name_and_info.second; if ((now >= info.next_update_time) && !info.loading() && info.loaded()) - should_update_map.emplace(info.object, info.hasException()); + should_update_map.emplace(info.object, info.failedToReload()); } } @@ -613,7 +613,7 @@ public: continue; } - /// Object was modified or it's loaded (possible outdated state) with exception, so it should be reloaded. + /// Object was modified or it was failed to reload last time, so it should be reloaded. startLoading(name, info); } else if (info.failed()) @@ -636,7 +636,7 @@ private: bool loading() const { return loading_id != 0; } bool wasLoading() const { return loaded() || failed() || loading(); } bool ready() const { return (loaded() || failed()) && !forced_to_reload; } - bool hasException() const { return exception != nullptr; } + bool failedToReload() const { return loaded() && exception != nullptr; } Status status() const { From 715d7a6455016a737cb264d8c516268674ff9ea0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 4 Dec 2019 20:24:53 +0300 Subject: [PATCH 42/77] Disable processors by default. --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 138f08f6bb5..06b77e22ca5 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -366,7 +366,7 @@ struct Settings : public SettingsCollection M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.", 0) \ \ - M(SettingBool, experimental_use_processors, true, "Use processors pipeline.", 0) \ + M(SettingBool, experimental_use_processors, false, "Use processors pipeline.", 0) \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.", 0) \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.", 0) \ From 172bbb1a2f74b1eec2599a59a4ab7cdb70e8dd35 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 4 Dec 2019 21:30:47 +0300 Subject: [PATCH 43/77] Fix flappy test, because dictionaries are loaded lazily --- .../queries/0_stateless/01033_dictionaries_lifetime.reference | 1 + dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql | 2 ++ 2 files changed, 3 insertions(+) diff --git a/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.reference b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.reference index b69b141bbe4..07c56f08482 100644 --- a/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.reference +++ b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.reference @@ -1,2 +1,3 @@ INITIALIZING DICTIONARY +1 1 10 diff --git a/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql index 0497349f86f..8b16c401afe 100644 --- a/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql +++ b/dbms/tests/queries/0_stateless/01033_dictionaries_lifetime.sql @@ -36,6 +36,8 @@ LAYOUT(FLAT()); SELECT 'INITIALIZING DICTIONARY'; +SELECT dictGetUInt8('ordinary_db.dict1', 'second_column', toUInt64(100500)); + SELECT lifetime_min, lifetime_max FROM system.dictionaries WHERE name = 'dict1'; DROP DICTIONARY IF EXISTS ordinary_db.dict1; From 3a72e1c12a2eb1bbd96750cfb1fb1f80b85f0169 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 10 Nov 2019 14:11:57 +0300 Subject: [PATCH 44/77] Add checksum for extra info/query in distributed sends This extras includes: - server revision - query settings - query Otherwise the code can try to interpret data, and got for instance std::length_error exception, which is not catched (to mark the part as broken). Also this will protect from the corruptions on disk. And add a simple test, since dbms/tests/integration/test_insert_into_distributed too complex. Also simplify the code by using readStringBinary() over readVarUInt()+b.readStrict() (this also gains additional checks that string is not bigger then 1GB). Refs: #4852 (8ef7f3589a5170ea6aae33872b3bfd815623db9e) v2: avoid ABI breakage (suggested by @vitlibar) v3: minor code fixes (suggested by @vitlibar) and as a consequence clang-8 build fix v4: drop DBMS_MIN_REVISION_WITH_EXTRAS_CHECKSUM_IN_DIST_BATCH and also revert some renames to make the patch cleaner --- .../Storages/Distributed/DirectoryMonitor.cpp | 48 +++++++++++++++++-- .../DistributedBlockOutputStream.cpp | 5 ++ ..._directory_monitor_batch_inserts.reference | 4 ++ ...ibuted_directory_monitor_batch_inserts.sql | 9 ++++ 4 files changed, 62 insertions(+), 4 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.reference create mode 100644 dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql diff --git a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp index 5a6f13b0567..21147417824 100644 --- a/dbms/src/Storages/Distributed/DirectoryMonitor.cpp +++ b/dbms/src/Storages/Distributed/DirectoryMonitor.cpp @@ -7,6 +7,8 @@ #include #include #include +#include +#include #include #include #include @@ -40,6 +42,7 @@ namespace ErrorCodes extern const int CHECKSUM_DOESNT_MATCH; extern const int TOO_LARGE_SIZE_COMPRESSED; extern const int ATTEMPT_TO_READ_AFTER_EOF; + extern const int CORRUPTED_DATA; } @@ -60,6 +63,19 @@ namespace return pools; } + + void assertChecksum(CityHash_v1_0_2::uint128 expected, CityHash_v1_0_2::uint128 calculated) + { + if (expected != calculated) + { + String message = "Checksum of extra info doesn't match: corrupted data." + " Reference: " + getHexUIntLowercase(expected.first) + getHexUIntLowercase(expected.second) + + ". Actual: " + getHexUIntLowercase(calculated.first) + getHexUIntLowercase(calculated.second) + + "."; + throw Exception(message, ErrorCodes::CHECKSUM_DOESNT_MATCH); + } + } + } @@ -277,13 +293,21 @@ void StorageDistributedDirectoryMonitor::readQueryAndSettings( if (query_size == DBMS_DISTRIBUTED_SIGNATURE_EXTRA_INFO) { + UInt64 initiator_revision; + CityHash_v1_0_2::uint128 expected; + CityHash_v1_0_2::uint128 calculated; + /// Read extra information. String extra_info_as_string; readStringBinary(extra_info_as_string, in); - readVarUInt(query_size, in); - ReadBufferFromString extra_info(extra_info_as_string); + /// To avoid out-of-bound, other cases will be checked in read*() helpers. + if (extra_info_as_string.size() < sizeof(expected)) + throw Exception("Not enough data", ErrorCodes::CORRUPTED_DATA); + + StringRef extra_info_ref(extra_info_as_string.data(), extra_info_as_string.size() - sizeof(expected)); + ReadBufferFromMemory extra_info(extra_info_ref.data, extra_info_ref.size); + ReadBuffer checksum(extra_info_as_string.data(), sizeof(expected), extra_info_ref.size); - UInt64 initiator_revision; readVarUInt(initiator_revision, extra_info); if (ClickHouseRevision::get() < initiator_revision) { @@ -293,13 +317,29 @@ void StorageDistributedDirectoryMonitor::readQueryAndSettings( << "It may lack support for new features."); } + /// Extra checksum (all data except itself -- this checksum) + readPODBinary(expected, checksum); + calculated = CityHash_v1_0_2::CityHash128(extra_info_ref.data, extra_info_ref.size); + assertChecksum(expected, calculated); + insert_settings.deserialize(extra_info); + /// Read query + readStringBinary(insert_query, in); + + /// Query checksum + readPODBinary(expected, extra_info); + calculated = CityHash_v1_0_2::CityHash128(insert_query.data(), insert_query.size()); + assertChecksum(expected, calculated); + /// Add handling new data here, for example: /// if (initiator_revision >= DBMS_MIN_REVISION_WITH_MY_NEW_DATA) /// readVarUInt(my_new_data, extra_info); + + return; } - else if (query_size == DBMS_DISTRIBUTED_SIGNATURE_SETTINGS_OLD_FORMAT) + + if (query_size == DBMS_DISTRIBUTED_SIGNATURE_SETTINGS_OLD_FORMAT) { insert_settings.deserialize(in, SettingsBinaryFormat::OLD); readVarUInt(query_size, in); diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index 61cb10cc38e..181909cff7a 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -595,9 +595,14 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: writeVarUInt(ClickHouseRevision::get(), extra_info); context.getSettingsRef().serialize(extra_info); + writePODBinary(CityHash_v1_0_2::CityHash128(query_string.data(), query_string.size()), extra_info); + /// Add new fields here, for example: /// writeVarUInt(my_new_data, extra_info); + const auto &extra_info_ref = extra_info.stringRef(); + writePODBinary(CityHash_v1_0_2::CityHash128(extra_info_ref.data, extra_info_ref.size), extra_info); + writeVarUInt(DBMS_DISTRIBUTED_SIGNATURE_EXTRA_INFO, out); writeStringBinary(extra_info.str(), out); diff --git a/dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.reference b/dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.reference new file mode 100644 index 00000000000..5565ed6787f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.reference @@ -0,0 +1,4 @@ +0 +1 +0 +1 diff --git a/dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql b/dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql new file mode 100644 index 00000000000..ffc33ce6949 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01040_distributed_directory_monitor_batch_inserts.sql @@ -0,0 +1,9 @@ +SET distributed_directory_monitor_batch_inserts=1; +SET distributed_directory_monitor_sleep_time_ms=10; +SET distributed_directory_monitor_max_sleep_time_ms=100; + +CREATE TABLE test (key UInt64) ENGINE=TinyLog(); +CREATE TABLE dist_test AS test Engine=Distributed(test_cluster_two_shards, currentDatabase(), test, key); +INSERT INTO dist_test SELECT toUInt64(number) FROM numbers(2); +SYSTEM FLUSH DISTRIBUTED dist_test; +SELECT * FROM dist_test; From d3d2e39955bd38ac4eea8be6b68c892e0314bd18 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 4 Dec 2019 23:23:07 +0300 Subject: [PATCH 45/77] fix SEMI RIGHT JOIN --- dbms/src/Interpreters/joinDispatch.h | 2 +- dbms/tests/queries/0_stateless/01031_semi_anti_join.reference | 1 + dbms/tests/queries/0_stateless/01031_semi_anti_join.sql | 2 +- 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/joinDispatch.h b/dbms/src/Interpreters/joinDispatch.h index 593a19ab637..a56b2ff3902 100644 --- a/dbms/src/Interpreters/joinDispatch.h +++ b/dbms/src/Interpreters/joinDispatch.h @@ -33,7 +33,7 @@ template <> struct MapGetter struct MapGetter { using Map = Join::MapsOne; }; template <> struct MapGetter { using Map = Join::MapsOne; }; -template <> struct MapGetter { using Map = Join::MapsAll; }; +template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; template <> struct MapGetter { using Map = Join::MapsOne; }; /// Only ANTI LEFT and ANTI RIGHT are valid diff --git a/dbms/tests/queries/0_stateless/01031_semi_anti_join.reference b/dbms/tests/queries/0_stateless/01031_semi_anti_join.reference index 5dbe67e50b4..782147f1f6f 100644 --- a/dbms/tests/queries/0_stateless/01031_semi_anti_join.reference +++ b/dbms/tests/queries/0_stateless/01031_semi_anti_join.reference @@ -1,5 +1,6 @@ semi left 2 a3 2 b1 +2 a6 2 b1 4 a5 4 b3 semi right 2 a3 2 b1 diff --git a/dbms/tests/queries/0_stateless/01031_semi_anti_join.sql b/dbms/tests/queries/0_stateless/01031_semi_anti_join.sql index ee3b81834df..19ea219563a 100644 --- a/dbms/tests/queries/0_stateless/01031_semi_anti_join.sql +++ b/dbms/tests/queries/0_stateless/01031_semi_anti_join.sql @@ -4,7 +4,7 @@ DROP TABLE IF EXISTS t2; CREATE TABLE t1 (x UInt32, s String) engine = Memory; CREATE TABLE t2 (x UInt32, s String) engine = Memory; -INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'); +INSERT INTO t1 (x, s) VALUES (0, 'a1'), (1, 'a2'), (2, 'a3'), (3, 'a4'), (4, 'a5'), (2, 'a6'); INSERT INTO t2 (x, s) VALUES (2, 'b1'), (2, 'b2'), (4, 'b3'), (4, 'b4'), (4, 'b5'), (5, 'b6'); SET join_use_nulls = 0; From 9fafadf53671359dd93f162498928b255f15f37c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 5 Dec 2019 05:17:46 +0300 Subject: [PATCH 46/77] Update extended_roadmap.md --- docs/ru/extended_roadmap.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/extended_roadmap.md b/docs/ru/extended_roadmap.md index 801a89af49e..85171cd93d7 100644 --- a/docs/ru/extended_roadmap.md +++ b/docs/ru/extended_roadmap.md @@ -1074,6 +1074,8 @@ Hold. Полезно для заказчиков внутри Яндекса, н ### 21.5. Распараллеливание INSERT при INSERT SELECT, если это необходимо. +[Vxider](https://github.com/Vxider), ICT + ### 21.6. Уменьшение числа потоков для SELECT в случае тривиального INSERT SELECT. ### 21.7. Кэш результатов запросов. From 9d005d156575d844e1d76ed75f7be3c4685570f5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 5 Dec 2019 12:49:30 +0300 Subject: [PATCH 47/77] true changes --- .../SummingSortedBlockInputStream.cpp | 8 ++++---- ...ncorrect_count_summing_merge_tree.reference | 18 +++++++++--------- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index 810fc9576ce..a3c376983f0 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -200,6 +200,10 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns) { + /// We have nothing to aggregate. It means that it could be non-zero, because we have columns_not_to_aggregate. + if (columns_to_aggregate.empty()) + current_row_is_zero = false; + for (auto & desc : columns_to_aggregate) { // Do not insert if the aggregation state hasn't been created @@ -329,11 +333,7 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: current_row_is_zero = true; } else - { key_differs = next_key != current_key; - /// If current_key is not empty - thats why current_row is not zero. - current_row_is_zero = false; - } if (key_differs) diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference index 6df56f0f19d..72ed0d6b3fd 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference @@ -3,20 +3,20 @@ 2018-02-02 00:00:00 2 -- 2 2 2 -1 +2 -- 2 2 2 -1 +2 -- 2 2 2 2 -- 2 2 2 -1 -1 -1 +2 +2 +2 -- 2 2 2 -1 +2 -- SummingMergeTree with Nullable column with duplicates 2018-02-01 00:00:00 4 2018-02-02 00:00:00 6 @@ -50,9 +50,9 @@ 2 2 -- 2 2 2 -1 -1 -1 +2 +2 +2 -- 2 2 2 2 From b6413d4a0454903dfda4e35f352aeb0eabfbb514 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 5 Dec 2019 12:52:49 +0300 Subject: [PATCH 48/77] better --- dbms/src/DataStreams/SummingSortedBlockInputStream.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index a3c376983f0..9ac7d6a3397 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -335,7 +335,6 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: else key_differs = next_key != current_key; - if (key_differs) { if (!current_key.empty()) From a3f3a333e2890e3617abf6e349b198b036e0ca4d Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 5 Dec 2019 12:55:06 +0300 Subject: [PATCH 49/77] better test --- .../01030_incorrect_count_summing_merge_tree.reference | 8 +++++--- .../01030_incorrect_count_summing_merge_tree.sql | 8 +++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference index 72ed0d6b3fd..b12baf4709e 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.reference @@ -10,7 +10,8 @@ -- 2 2 2 2 --- 2 2 2 +-- 2 2 2 2 +2 2 2 2 @@ -49,7 +50,8 @@ -- 2 2 2 2 --- 2 2 2 +-- 2 2 2 2 +2 2 2 2 @@ -68,7 +70,7 @@ -- 4 2 4 2 --- 2 2 2 +-- 2 2 2 2 2 2 2 diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql index 805af881480..a9f7bf7ecd7 100644 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql +++ b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree.sql @@ -18,10 +18,11 @@ select '-- 2 2'; select count() from tst where val is not null; select count() from tst final where val is not null; -select '-- 2 2 2'; +select '-- 2 2 2 2'; select count() from tst final where timestamp>0; select count() from tst final prewhere timestamp > 0; select count() from tst final where timestamp > '2017-01-01 00:00:00'; +select count() from tst final prewhere timestamp > '2017-01-01 00:00:00'; select '-- 2 2'; select count() from tst final where val>0; @@ -77,10 +78,11 @@ select '-- 2 2'; select count() from tst where val is not null; select count() from tst final where val is not null; -select '-- 2 2 2'; +select '-- 2 2 2 2'; select count() from tst final where timestamp>0; select count() from tst final prewhere timestamp > 0; select count() from tst final where timestamp > '2017-01-01 00:00:00'; +select count() from tst final prewhere timestamp > '2017-01-01 00:00:00'; select '-- 2 2'; select count() from tst final where val>0; @@ -108,7 +110,7 @@ select '-- 4 2'; select count() from tst where val is not null; select count() from tst final where val is not null; -select '-- 2 2 2'; +select '-- 2 2 2 2'; select count() from tst final where timestamp>0; select count() from tst final prewhere timestamp > 0; select count() from tst final where timestamp > '2017-01-01 00:00:00'; From 529293faad876712c247c603ed515befd994128e Mon Sep 17 00:00:00 2001 From: Sergei Bocharov Date: Thu, 5 Dec 2019 13:03:35 +0300 Subject: [PATCH 50/77] Docs: added docs for trimLeft, trimRight, trimBoth (#7924) --- .../functions/string_functions.md | 114 +++++++++++++++++- .../functions/string_functions.md | 114 ++++++++++++++++++ 2 files changed, 222 insertions(+), 6 deletions(-) diff --git a/docs/en/query_language/functions/string_functions.md b/docs/en/query_language/functions/string_functions.md index a45f41a4528..33e5700f355 100644 --- a/docs/en/query_language/functions/string_functions.md +++ b/docs/en/query_language/functions/string_functions.md @@ -217,17 +217,119 @@ Result: └───────────────────────────────────┘ ``` -## trimLeft(s) +## trimLeft {#trimleft} -Returns a string that removes the whitespace characters on left side. +Removes all consecutive occurrences of common whitespace (ASCII character 32) from the beginning of a string. It doesn't remove other kinds of whitespace characters (tab, no-break space, etc.). -## trimRight(s) +**Syntax** -Returns a string that removes the whitespace characters on right side. +```sql +trimLeft() +``` -## trimBoth(s) +Alias: `ltrim`. -Returns a string that removes the whitespace characters on either side. +**Parameters** + +- `string` — string to trim. [String](../../data_types/string.md). + +**Returned value** + +A string without leading common whitespaces. + +Type: `String`. + +**Example** + +Query: + +```sql +SELECT trimLeft(' Hello, world! ') +``` + +Result: + +```text +┌─trimLeft(' Hello, world! ')─┐ +│ Hello, world! │ +└─────────────────────────────────────┘ +``` + +## trimRight {#trimright} + +Removes all consecutive occurrences of common whitespace (ASCII character 32) from the end of a string. It doesn't remove other kinds of whitespace characters (tab, no-break space, etc.). + +**Syntax** + +```sql +trimRight() +``` + +Alias: `rtrim`. + +**Parameters** + +- `string` — string to trim. [String](../../data_types/string.md). + +**Returned value** + +A string without trailing common whitespaces. + +Type: `String`. + +**Example** + +Query: + +```sql +SELECT trimRight(' Hello, world! ') +``` + +Result: + +```text +┌─trimRight(' Hello, world! ')─┐ +│ Hello, world! │ +└──────────────────────────────────────┘ +``` + +## trimBoth {#trimboth} + +Removes all consecutive occurrences of common whitespace (ASCII character 32) from both ends of a string. It doesn't remove other kinds of whitespace characters (tab, no-break space, etc.). + +**Syntax** + +```sql +trimBoth() +``` + +Alias: `trim`. + +**Parameters** + +- `string` — string to trim. [String](../../data_types/string.md). + +**Returned value** + +A string without leading and trailing common whitespaces. + +Type: `String`. + +**Example** + +Query: + +```sql +SELECT trimBoth(' Hello, world! ') +``` + +Result: + +```text +┌─trimBoth(' Hello, world! ')─┐ +│ Hello, world! │ +└─────────────────────────────────────┘ +``` ## CRC32(s) diff --git a/docs/ru/query_language/functions/string_functions.md b/docs/ru/query_language/functions/string_functions.md index 5e5a270f51b..2169cb794e0 100644 --- a/docs/ru/query_language/functions/string_functions.md +++ b/docs/ru/query_language/functions/string_functions.md @@ -189,6 +189,120 @@ SELECT startsWith('Hello, world!', 'He'); └───────────────────────────────────┘ ``` +## trimLeft {#trimleft} + +Удаляет все последовательные вхождения обычных пробелов (32 символ ASCII) с левого конца строки. Не удаляет другие виды пробелов (табуляция, пробел без разрыва и т. д.). + +**Синтаксис** + +```sql +trimLeft() +``` + +Алиас: `ltrim`. + +**Параметры** + +- `string` — строка для обрезки. [String](../../data_types/string.md). + +**Возвращаемое значение** + +Исходную строку без общих пробельных символов слева. + +Тип: `String`. + +**Пример** + +Запрос: + +```sql +SELECT trimLeft(' Hello, world! ') +``` + +Ответ: + +```text +┌─trimLeft(' Hello, world! ')─┐ +│ Hello, world! │ +└─────────────────────────────────────┘ +``` + +## trimRight {#trimright} + +Удаляет все последовательные вхождения обычных пробелов (32 символ ASCII) с правого конца строки. Не удаляет другие виды пробелов (табуляция, пробел без разрыва и т. д.). + +**Синтаксис** + +```sql +trimRight() +``` + +Алиас: `rtrim`. + +**Параметры** + +- `string` — строка для обрезки. [String](../../data_types/string.md). + +**Возвращаемое значение** + +Исходную строку без общих пробельных символов справа. + +Тип: `String`. + +**Пример** + +Запрос: + +```sql +SELECT trimRight(' Hello, world! ') +``` + +Ответ: + +```text +┌─trimRight(' Hello, world! ')─┐ +│ Hello, world! │ +└──────────────────────────────────────┘ +``` + +## trimBoth {#trimboth} + +Удаляет все последовательные вхождения обычных пробелов (32 символ ASCII) с обоих концов строки. Не удаляет другие виды пробелов (табуляция, пробел без разрыва и т. д.). + +**Синтаксис** + +```sql +trimBoth() +``` + +Алиас: `trim`. + +**Параметры** + +- `string` — строка для обрезки. [String](../../data_types/string.md). + +**Возвращаемое значение** + +Исходную строку без общих пробельных символов с обоих концов строки. + +Тип: `String`. + +**Пример** + +Запрос: + +```sql +SELECT trimBoth(' Hello, world! ') +``` + +Ответ: + +```text +┌─trimBoth(' Hello, world! ')─┐ +│ Hello, world! │ +└─────────────────────────────────────┘ +``` + ## CRC32(s) Возвращает чексумму CRC32 данной строки, используется CRC-32-IEEE 802.3 многочлен и начальным значением `0xffffffff` (т.к. используется реализация из zlib). From 88e37020e081d56985974a3494d103566f3abdd0 Mon Sep 17 00:00:00 2001 From: Mikhail Korotov <55493615+millb@users.noreply.github.com> Date: Thu, 5 Dec 2019 13:13:40 +0300 Subject: [PATCH 51/77] JSONCompactEachRow and JSONCompactEachRowWithNamesAndTypes (#7841) * Research commit * Created Output EachRow Format * Fixed bugs * Created output format JSONCompactEachRowWithNamesAndTypes without totals * Fixed bugs * Fixed bugs * Totals for JSONCompactEachRowWithNamesAndTypes * Deleted needless debug * Working commit * Working commit * Working commit * Working commit * Working commit * Working commit * Working commit * Working commit * Working commit * Working commit * Working commit * Working commit * Tests added * Input Format for JSONCompactEachRow * Fixed bugs for JSONCompactEachRow Input format * Fixed bugs for JSONCompactEachRowRowInputFormat.cpp * JSONCompactEachRow and JSONCompactEachRowWithNamesAndTypes united * Created input format for both formats * fixed bugs * fixed bugs * Working commit * Working commit * Working commit * Working commit * tests * Working commit * Final tests * Performance tests added --- dbms/src/Formats/FormatFactory.cpp | 4 + .../Impl/JSONCompactEachRowRowInputFormat.cpp | 238 ++++++++++++++++++ .../Impl/JSONCompactEachRowRowInputFormat.h | 54 ++++ .../JSONCompactEachRowRowOutputFormat.cpp | 116 +++++++++ .../Impl/JSONCompactEachRowRowOutputFormat.h | 45 ++++ dbms/tests/performance/parse_engine_file.xml | 2 + dbms/tests/performance/select_format.xml | 1 + .../01034_JSONCompactEachRow.reference | 47 ++++ .../0_stateless/01034_JSONCompactEachRow.sql | 63 +++++ 9 files changed, 570 insertions(+) create mode 100644 dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp create mode 100644 dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h create mode 100644 dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp create mode 100644 dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h create mode 100644 dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.reference create mode 100644 dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.sql diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 57071b17c28..dfbaef334e0 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -281,6 +281,8 @@ void registerInputFormatProcessorTSKV(FormatFactory & factory); void registerOutputFormatProcessorTSKV(FormatFactory & factory); void registerInputFormatProcessorJSONEachRow(FormatFactory & factory); void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory); +void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory); +void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory); void registerInputFormatProcessorParquet(FormatFactory & factory); void registerInputFormatProcessorORC(FormatFactory & factory); void registerOutputFormatProcessorParquet(FormatFactory & factory); @@ -336,6 +338,8 @@ FormatFactory::FormatFactory() registerOutputFormatProcessorTSKV(*this); registerInputFormatProcessorJSONEachRow(*this); registerOutputFormatProcessorJSONEachRow(*this); + registerInputFormatProcessorJSONCompactEachRow(*this); + registerOutputFormatProcessorJSONCompactEachRow(*this); registerInputFormatProcessorProtobuf(*this); registerOutputFormatProcessorProtobuf(*this); registerInputFormatProcessorCapnProto(*this); diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp new file mode 100644 index 00000000000..d4530e7b09d --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.cpp @@ -0,0 +1,238 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INCORRECT_DATA; + extern const int CANNOT_READ_ALL_DATA; + extern const int LOGICAL_ERROR; +} + + +JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(ReadBuffer & in_, + const Block & header_, + Params params_, + const FormatSettings & format_settings_, + bool with_names_) + : IRowInputFormat(header_, in_, std::move(params_)), format_settings(format_settings_), with_names(with_names_) +{ + /// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it. + skipBOMIfExists(in); + auto & sample = getPort().getHeader(); + size_t num_columns = sample.columns(); + + data_types.resize(num_columns); + column_indexes_by_names.reserve(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + const auto & column_info = sample.getByPosition(i); + + data_types[i] = column_info.type; + column_indexes_by_names.emplace(column_info.name, i); + } +} + +void JSONCompactEachRowRowInputFormat::readPrefix() +{ + if (with_names) + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, false); + + assertChar('[', in); + do + { + skipWhitespaceIfAny(in); + String column_name; + readJSONString(column_name, in); + addInputColumn(column_name); + skipWhitespaceIfAny(in); + } + while (checkChar(',', in)); + assertChar(']', in); + skipEndOfLine(); + + /// Type checking + assertChar('[', in); + for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) + { + skipWhitespaceIfAny(in); + String data_type; + readJSONString(data_type, in); + + if (column_indexes_for_input_fields[i] && + data_types[*column_indexes_for_input_fields[i]]->getName() != data_type) + { + throw Exception( + "Type of '" + getPort().getHeader().getByPosition(*column_indexes_for_input_fields[i]).name + + "' must be " + data_types[*column_indexes_for_input_fields[i]]->getName() + + ", not " + data_type, + ErrorCodes::INCORRECT_DATA + ); + } + + if (i != column_indexes_for_input_fields.size() - 1) + assertChar(',', in); + skipWhitespaceIfAny(in); + } + assertChar(']', in); + } + else + { + size_t num_columns = getPort().getHeader().columns(); + read_columns.assign(num_columns, true); + column_indexes_for_input_fields.resize(num_columns); + + for (size_t i = 0; i < num_columns; ++i) + { + column_indexes_for_input_fields[i] = i; + } + } + + for (size_t i = 0; i < read_columns.size(); ++i) + { + if (!read_columns[i]) + { + not_seen_columns.emplace_back(i); + } + } +} + +void JSONCompactEachRowRowInputFormat::addInputColumn(const String & column_name) +{ + names_of_columns.emplace_back(column_name); + + const auto column_it = column_indexes_by_names.find(column_name); + if (column_it == column_indexes_by_names.end()) + { + if (format_settings.skip_unknown_fields) + { + column_indexes_for_input_fields.push_back(std::nullopt); + return; + } + + throw Exception( + "Unknown field found in JSONCompactEachRow header: '" + column_name + "' " + + "at position " + std::to_string(column_indexes_for_input_fields.size()) + + "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", + ErrorCodes::INCORRECT_DATA + ); + } + + const auto column_index = column_it->second; + + if (read_columns[column_index]) + throw Exception("Duplicate field found while parsing JSONCompactEachRow header: " + column_name, ErrorCodes::INCORRECT_DATA); + + read_columns[column_index] = true; + column_indexes_for_input_fields.emplace_back(column_index); +} + +bool JSONCompactEachRowRowInputFormat::readRow(DB::MutableColumns &columns, DB::RowReadExtension &ext) +{ + skipEndOfLine(); + + if (in.eof()) + return false; + + size_t num_columns = columns.size(); + + read_columns.assign(num_columns, false); + + assertChar('[', in); + for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) + { + const auto & table_column = column_indexes_for_input_fields[file_column]; + if (table_column) + { + readField(*table_column, columns); + } + else + { + skipJSONField(in, StringRef(names_of_columns[file_column])); + } + + skipWhitespaceIfAny(in); + if (in.eof()) + throw Exception("Unexpected end of stream while parsing JSONCompactEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA); + if (file_column + 1 != column_indexes_for_input_fields.size()) + { + assertChar(',', in); + skipWhitespaceIfAny(in); + } + } + assertChar(']', in); + + for (size_t i = 0; i < not_seen_columns.size(); i++) + { + columns[not_seen_columns[i]]->insertDefault(); + } + + ext.read_columns = read_columns; + return true; +} + +void JSONCompactEachRowRowInputFormat::skipEndOfLine() +{ + skipWhitespaceIfAny(in); + if (!in.eof() && (*in.position() == ',' || *in.position() == ';')) + ++in.position(); + + skipWhitespaceIfAny(in); +} + +void JSONCompactEachRowRowInputFormat::readField(size_t index, MutableColumns & columns) +{ + try + { + read_columns[index] = true; + const auto & type = data_types[index]; + if (format_settings.null_as_default && !type->isNullable()) + read_columns[index] = DataTypeNullable::deserializeTextJSON(*columns[index], in, format_settings, type); + else + type->deserializeAsTextJSON(*columns[index], in, format_settings); + } + catch (Exception & e) + { + e.addMessage("(while read the value of key " + getPort().getHeader().getByPosition(index).name + ")"); + throw; + } +} + +void JSONCompactEachRowRowInputFormat::syncAfterError() +{ + skipToUnescapedNextLineOrEOF(in); +} + +void registerInputFormatProcessorJSONCompactEachRow(FormatFactory & factory) +{ + factory.registerInputFormatProcessor("JSONCompactEachRow", []( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, false); + }); + + factory.registerInputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(buf, sample, std::move(params), settings, true); + }); +} + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h new file mode 100644 index 00000000000..e633475d0f4 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -0,0 +1,54 @@ +#pragma once + +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ReadBuffer; + +/** A stream for reading data in JSONCompactEachRow and JSONCompactEachRowWithNamesAndTypes formats +*/ +class JSONCompactEachRowRowInputFormat : public IRowInputFormat +{ +public: + JSONCompactEachRowRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_, bool with_names_); + + String getName() const override { return "JSONCompactEachRowRowInputFormat"; } + + + void readPrefix() override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool allowSyncAfterError() const override { return true; } + void syncAfterError() override; + + +private: + void addInputColumn(const String & column_name); + void skipEndOfLine(); + void readField(size_t index, MutableColumns & columns); + + const FormatSettings format_settings; + + using IndexesMap = std::unordered_map; + IndexesMap column_indexes_by_names; + + using OptionalIndexes = std::vector>; + OptionalIndexes column_indexes_for_input_fields; + + DataTypes data_types; + std::vector read_columns; + std::vector not_seen_columns; + + /// This is for the correct exceptions in skipping unknown fields. + std::vector names_of_columns; + + bool with_names; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp new file mode 100644 index 00000000000..433cc4515ae --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -0,0 +1,116 @@ +#include +#include +#include +#include + + +namespace DB +{ + + +JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, + const Block & header_, + FormatFactory::WriteCallback callback, + const FormatSettings & settings_, + bool with_names_) + : IRowOutputFormat(header_, out_, callback), settings(settings_), with_names(with_names_) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + NamesAndTypesList columns(sample.getNamesAndTypesList()); + fields.assign(columns.begin(), columns.end()); +} + + +void JSONCompactEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num) +{ + type.serializeAsTextJSON(column, row_num, out, settings); +} + + +void JSONCompactEachRowRowOutputFormat::writeFieldDelimiter() +{ + writeCString(", ", out); +} + + +void JSONCompactEachRowRowOutputFormat::writeRowStartDelimiter() +{ + writeChar('[', out); +} + + +void JSONCompactEachRowRowOutputFormat::writeRowEndDelimiter() +{ + writeCString("]\n", out); +} + +void JSONCompactEachRowRowOutputFormat::writeTotals(const Columns & columns, size_t row_num) +{ + writeChar('\n', out); + size_t num_columns = columns.size(); + writeChar('[', out); + for (size_t i = 0; i < num_columns; ++i) + { + if (i != 0) + JSONCompactEachRowRowOutputFormat::writeFieldDelimiter(); + + JSONCompactEachRowRowOutputFormat::writeField(*columns[i], *types[i], row_num); + } + writeCString("]\n", out); +} + +void JSONCompactEachRowRowOutputFormat::writePrefix() +{ + if (with_names) + { + writeChar('[', out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeChar('\"', out); + writeString(fields[i].name, out); + writeChar('\"', out); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n[", out); + for (size_t i = 0; i < fields.size(); ++i) + { + writeJSONString(fields[i].type->getName(), out, settings); + if (i != fields.size() - 1) + writeCString(", ", out); + } + writeCString("]\n", out); + } +} + +void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk) +{ + if (with_names) + IRowOutputFormat::consumeTotals(std::move(chunk)); +} + +void registerOutputFormatProcessorJSONCompactEachRow(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("JSONCompactEachRow", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + FormatFactory::WriteCallback callback, + const FormatSettings & format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, false); + }); + + factory.registerOutputFormatProcessor("JSONCompactEachRowWithNamesAndTypes", []( + WriteBuffer &buf, + const Block &sample, + const Context &, + FormatFactory::WriteCallback callback, + const FormatSettings &format_settings) + { + return std::make_shared(buf, sample, callback, format_settings, true); + }); +} + + +} diff --git a/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h new file mode 100644 index 00000000000..a7857a82d2d --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -0,0 +1,45 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/** The stream for outputting data in JSON format, by object per line. + * Does not validate UTF-8. + */ +class JSONCompactEachRowRowOutputFormat : public IRowOutputFormat +{ +public: + JSONCompactEachRowRowOutputFormat(WriteBuffer & out_, const Block & header_, FormatFactory::WriteCallback callback, const FormatSettings & settings_, bool with_names); + + String getName() const override { return "JSONCompactEachRowRowOutputFormat"; } + + void writePrefix() override; + + void writeBeforeTotals() override {} + void writeTotals(const Columns & columns, size_t row_num) override; + void writeAfterTotals() override {} + + void writeField(const IColumn & column, const IDataType & type, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + +protected: + void consumeTotals(Chunk) override; + /// No extremes. + void consumeExtremes(Chunk) override {} + +private: + FormatSettings settings; + + NamesAndTypes fields; + + bool with_names; +}; +} diff --git a/dbms/tests/performance/parse_engine_file.xml b/dbms/tests/performance/parse_engine_file.xml index 6bd4af0b45b..8308d8f049f 100644 --- a/dbms/tests/performance/parse_engine_file.xml +++ b/dbms/tests/performance/parse_engine_file.xml @@ -32,6 +32,8 @@ CSVWithNames Values JSONEachRow + JSONCompactEachRow + JSONCompactEachRowWithNamesAndTypes TSKV RowBinary Native diff --git a/dbms/tests/performance/select_format.xml b/dbms/tests/performance/select_format.xml index c5ad1acd396..55ab7b2d458 100644 --- a/dbms/tests/performance/select_format.xml +++ b/dbms/tests/performance/select_format.xml @@ -34,6 +34,7 @@ JSON JSONCompact JSONEachRow + JSONCompactEachRow TSKV Pretty PrettyCompact diff --git a/dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.reference b/dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.reference new file mode 100644 index 00000000000..6ec53e11fc9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.reference @@ -0,0 +1,47 @@ +1 +[1, "a"] +[2, "b"] +[3, "c"] +2 +["a", "1"] +["b", "1"] +["c", "1"] +3 +["value", "name"] +["UInt8", "String"] +[1, "a"] +[2, "b"] +[3, "c"] +4 +["name", "c"] +["String", "UInt64"] +["a", "1"] +["b", "1"] +["c", "1"] + +["", "3"] +5 +["first", 1, 2, 0] +["second", 2, 0, 6] +6 +["first", 1, 2, 8] +["second", 2, 32, 6] +7 +[16, [15,16,0], ["first","second","third"]] +8 +["first", 1, 2, 0] +["second", 2, 0, 6] +9 +["first", 1, 2, 8] +["second", 2, 32, 6] +10 +["first", 1, 16, 8] +["second", 2, 32, 8] +11 +["v1", "v2", "v3", "v4"] +["String", "UInt8", "UInt16", "UInt8"] +["", 2, 3, 1] +12 +["v1", "n.id", "n.name"] +["UInt8", "Array(UInt8)", "Array(String)"] +[16, [15,16,0], ["first","second","third"]] diff --git a/dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.sql b/dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.sql new file mode 100644 index 00000000000..46a0e90e69d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01034_JSONCompactEachRow.sql @@ -0,0 +1,63 @@ +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; +SELECT 1; +/* Check JSONCompactEachRow Output */ +CREATE TABLE test_table (value UInt8, name String) ENGINE = MergeTree() ORDER BY value; +INSERT INTO test_table VALUES (1, 'a'), (2, 'b'), (3, 'c'); +SELECT * FROM test_table FORMAT JSONCompactEachRow; +SELECT 2; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactEachRow; +SELECT 3; +/* Check JSONCompactEachRowWithNamesAndTypes Output */ +SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes; +SELECT 4; +/* Check Totals */ +SELECT name, count() AS c FROM test_table GROUP BY name WITH TOTALS ORDER BY name FORMAT JSONCompactEachRowWithNamesAndTypes; +DROP TABLE IF EXISTS test_table; +SELECT 5; +/* Check JSONCompactEachRow Input */ +CREATE TABLE test_table (v1 String, v2 UInt8, v3 DEFAULT v2 * 16, v4 UInt8 DEFAULT 8) ENGINE = MergeTree() ORDER BY v2; +INSERT INTO test_table FORMAT JSONCompactEachRow ["first", 1, "2", null] ["second", 2, null, 6]; +SELECT * FROM test_table FORMAT JSONCompactEachRow; +TRUNCATE TABLE test_table; +SELECT 6; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactEachRow ["first", 1, "2", null] ["second", 2, null, 6]; +SELECT * FROM test_table FORMAT JSONCompactEachRow; +TRUNCATE TABLE test_table; +SELECT 7; +/* Check Nested */ +CREATE TABLE test_table_2 (v1 UInt8, n Nested(id UInt8, name String)) ENGINE = MergeTree() ORDER BY v1; +INSERT INTO test_table_2 FORMAT JSONCompactEachRow [16, [15, 16, null], ["first", "second", "third"]]; +SELECT * FROM test_table_2 FORMAT JSONCompactEachRow; +TRUNCATE TABLE test_table_2; +SELECT 8; +/* Check JSONCompactEachRowWithNamesAndTypes Output */ +SET input_format_null_as_default = 0; +INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null]["second", 2, null, 6]; +SELECT * FROM test_table FORMAT JSONCompactEachRow; +TRUNCATE TABLE test_table; +SELECT 9; +/* Check input_format_null_as_default = 1 */ +SET input_format_null_as_default = 1; +INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "v3", "v4"]["String","UInt8","UInt16","UInt8"]["first", 1, "2", null] ["second", 2, null, 6]; +SELECT * FROM test_table FORMAT JSONCompactEachRow; +SELECT 10; +/* Check Header */ +TRUNCATE TABLE test_table; +SET input_format_skip_unknown_fields = 1; +INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "v2", "invalid_column"]["String", "UInt8", "UInt8"]["first", 1, 32]["second", 2, "64"]; +SELECT * FROM test_table FORMAT JSONCompactEachRow; +SELECT 11; +TRUNCATE TABLE test_table; +INSERT INTO test_table FORMAT JSONCompactEachRowWithNamesAndTypes ["v4", "v2", "v3"]["UInt8", "UInt8", "UInt16"][1, 2, 3] +SELECT * FROM test_table FORMAT JSONCompactEachRowWithNamesAndTypes; +SELECT 12; +/* Check Nested */ +INSERT INTO test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes ["v1", "n.id", "n.name"]["UInt8", "Array(UInt8)", "Array(String)"][16, [15, 16, null], ["first", "second", "third"]]; +SELECT * FROM test_table_2 FORMAT JSONCompactEachRowWithNamesAndTypes; + +DROP TABLE IF EXISTS test_table; +DROP TABLE IF EXISTS test_table_2; From 852e891499cd3b9f1dbcdfec6c9e28ddb5ff7548 Mon Sep 17 00:00:00 2001 From: chertus Date: Thu, 5 Dec 2019 15:01:34 +0300 Subject: [PATCH 52/77] better comment --- dbms/src/Interpreters/joinDispatch.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/joinDispatch.h b/dbms/src/Interpreters/joinDispatch.h index a56b2ff3902..840b9b91a66 100644 --- a/dbms/src/Interpreters/joinDispatch.h +++ b/dbms/src/Interpreters/joinDispatch.h @@ -30,13 +30,13 @@ template <> struct MapGetter struct MapGetter { using Map = Join::MapsAllFlagged; }; template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; -/// Only SEMI LEFT and SEMI RIGHT are valid +/// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. template <> struct MapGetter { using Map = Join::MapsOne; }; template <> struct MapGetter { using Map = Join::MapsOne; }; template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; template <> struct MapGetter { using Map = Join::MapsOne; }; -/// Only ANTI LEFT and ANTI RIGHT are valid +/// Only SEMI LEFT and SEMI RIGHT are valid. INNER and FULL are here for templates instantiation. template <> struct MapGetter { using Map = Join::MapsOne; }; template <> struct MapGetter { using Map = Join::MapsOne; }; template <> struct MapGetter { using Map = Join::MapsAllFlagged; }; From ee46a73d257820d362555b058dba000692a588ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 5 Dec 2019 15:01:36 +0300 Subject: [PATCH 53/77] Fix bug in checkDataPart when last granule has same number of rows as in .mrk file --- .../MergeTree/MergeTreeIndexGranularity.h | 3 +++ dbms/src/Storages/MergeTree/checkDataPart.cpp | 5 ++++- ...2_check_query_and_last_granule_size.reference | 2 ++ .../01042_check_query_and_last_granule_size.sql | 16 ++++++++++++++++ 4 files changed, 25 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference create mode 100644 dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql diff --git a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularity.h b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularity.h index ff391be596c..0c76f74c3df 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeIndexGranularity.h +++ b/dbms/src/Storages/MergeTree/MergeTreeIndexGranularity.h @@ -43,6 +43,9 @@ public: /// Total rows size_t getTotalRows() const; + /// Total number marks without final mark if it exists + size_t getMarksCountWithoutFinal() const { return getMarksCount() - hasFinalMark(); } + /// Rows after mark to next mark inline size_t getMarkRows(size_t mark_index) const { diff --git a/dbms/src/Storages/MergeTree/checkDataPart.cpp b/dbms/src/Storages/MergeTree/checkDataPart.cpp index 092cc78e313..a2d6a836d6f 100644 --- a/dbms/src/Storages/MergeTree/checkDataPart.cpp +++ b/dbms/src/Storages/MergeTree/checkDataPart.cpp @@ -143,12 +143,14 @@ public: + toString(compressed_hashing_buf.count()) + " (compressed), " + toString(uncompressed_hashing_buf.count()) + " (uncompressed)", ErrorCodes::CORRUPTED_DATA); + /// Maybe we have final mark. if (index_granularity.hasFinalMark()) { auto final_mark_rows = readMarkFromFile().second; if (final_mark_rows != 0) throw Exception("Incorrect final mark at the end of " + mrk_file_path + " expected 0 rows, got " + toString(final_mark_rows), ErrorCodes::CORRUPTED_DATA); } + if (!mrk_hashing_buf.eof()) throw Exception("EOF expected in " + mrk_file_path + " file" + " at position " @@ -379,7 +381,8 @@ MergeTreeData::DataPart::Checksums checkDataPart( size_t read_size = tmp_column->size(); column_size += read_size; - if (read_size < rows_after_mark || mark_num == adaptive_index_granularity.getMarksCount()) + /// We already checked all marks except final (it will be checked in assertEnd()). + if (mark_num == adaptive_index_granularity.getMarksCountWithoutFinal()) break; else if (marks_eof) throw Exception("Unexpected end of mrk file while reading column " + name_type.name, ErrorCodes::CORRUPTED_DATA); diff --git a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference new file mode 100644 index 00000000000..3025e6463d8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference @@ -0,0 +1,2 @@ +all_1_1_0 1 +all_1_1_0 1 diff --git a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql new file mode 100644 index 00000000000..8061bd64d1f --- /dev/null +++ b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -0,0 +1,16 @@ +SET check_query_single_value_result = 0; +DROP TABLE IF EXISTS check_query_test; + +CREATE TABLE check_query_test (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey; + +-- Number of rows in last granule should be equals to granularity. +-- Rows in this table are short, so granularity will be 8192. +INSERT INTO check_query_test SELECT number, toString(number) FROM system.numbers LIMIT 81920; + +CHECK TABLE check_query_test; + +OPTIMIZE TABLE check_query_test; + +CHECK TABLE check_query_test; + +DROP TABLE IF EXISTS check_query_test; From cd43debd798318acf7fb68518d9fd603944cf5ac Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 5 Dec 2019 15:18:18 +0300 Subject: [PATCH 54/77] Test with non adaptive parts --- ...042_check_query_and_last_granule_size.reference | 2 ++ .../01042_check_query_and_last_granule_size.sql | 14 ++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference index 3025e6463d8..f780cabf175 100644 --- a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference +++ b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference @@ -1,2 +1,4 @@ all_1_1_0 1 all_1_1_0 1 +all_1_1_0 1 +all_1_1_0 1 diff --git a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index 8061bd64d1f..4d7b6adfaa7 100644 --- a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -14,3 +14,17 @@ OPTIMIZE TABLE check_query_test; CHECK TABLE check_query_test; DROP TABLE IF EXISTS check_query_test; + +DROP TABLE IF EXISTS check_query_test_non_adaptive; + +CREATE TABLE check_query_test_non_adaptive (SomeKey UInt64, SomeValue String) ENGINE = MergeTree() ORDER BY SomeKey SETTINGS index_granularity_bytes = 0; + +INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 81920; + +CHECK TABLE check_query_test_non_adaptive; + +OPTIMIZE TABLE check_query_test_non_adaptive; + +CHECK TABLE check_query_test_non_adaptive; + +DROP TABLE IF EXISTS check_query_test_non_adaptive; From 227d0ba9d6e9ccda59938efa50bc9d99b9f2a5ae Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 5 Dec 2019 15:39:02 +0300 Subject: [PATCH 55/77] Add one more test --- .../01042_check_query_and_last_granule_size.reference | 3 +++ .../01042_check_query_and_last_granule_size.sql | 8 ++++++++ 2 files changed, 11 insertions(+) diff --git a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference index f780cabf175..a4fac158712 100644 --- a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference +++ b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.reference @@ -2,3 +2,6 @@ all_1_1_0 1 all_1_1_0 1 all_1_1_0 1 all_1_1_0 1 +all_1_1_0 1 +all_2_2_0 1 +all_1_2_1 1 diff --git a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql index 4d7b6adfaa7..9777ea1dc45 100644 --- a/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql +++ b/dbms/tests/queries/0_stateless/01042_check_query_and_last_granule_size.sql @@ -27,4 +27,12 @@ OPTIMIZE TABLE check_query_test_non_adaptive; CHECK TABLE check_query_test_non_adaptive; +INSERT INTO check_query_test_non_adaptive SELECT number, toString(number) FROM system.numbers LIMIT 77; + +CHECK TABLE check_query_test_non_adaptive; + +OPTIMIZE TABLE check_query_test_non_adaptive; + +CHECK TABLE check_query_test_non_adaptive; + DROP TABLE IF EXISTS check_query_test_non_adaptive; From 4f7497449208fac2769219c3ca23619168ce5d5a Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 5 Dec 2019 17:34:35 +0300 Subject: [PATCH 56/77] fixed warning unique_lock --- .../DataStreams/ParallelParsingBlockInputStream.cpp | 13 ++++++------- .../DataStreams/ParallelParsingBlockInputStream.h | 4 ++-- 2 files changed, 8 insertions(+), 9 deletions(-) diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp index 21233da9327..c894af82580 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp @@ -1,5 +1,4 @@ #include -#include "ParallelParsingBlockInputStream.h" namespace DB { @@ -15,7 +14,7 @@ void ParallelParsingBlockInputStream::segmentatorThreadFunction() auto & unit = processing_units[current_unit_number]; { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); segmentator_condvar.wait(lock, [&]{ return unit.status == READY_TO_INSERT || finished; }); } @@ -85,7 +84,7 @@ void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_n // except at the end of file. Also see a matching assert in readImpl(). assert(unit.is_last || unit.block_ext.block.size() > 0); - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); unit.status = READY_TO_READ; reader_condvar.notify_all(); } @@ -99,7 +98,7 @@ void ParallelParsingBlockInputStream::onBackgroundException() { tryLogCurrentException(__PRETTY_FUNCTION__); - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); if (!background_exception) { background_exception = std::current_exception(); @@ -116,7 +115,7 @@ Block ParallelParsingBlockInputStream::readImpl() /** * Check for background exception and rethrow it before we return. */ - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); if (background_exception) { lock.unlock(); @@ -134,7 +133,7 @@ Block ParallelParsingBlockInputStream::readImpl() { // We have read out all the Blocks from the previous Processing Unit, // wait for the current one to become ready. - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || finished; }); if (finished) @@ -190,7 +189,7 @@ Block ParallelParsingBlockInputStream::readImpl() else { // Pass the unit back to the segmentator. - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); unit.status = READY_TO_INSERT; segmentator_condvar.notify_all(); } diff --git a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h index 4b5e091cfc9..8c276f2f7dd 100644 --- a/dbms/src/DataStreams/ParallelParsingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelParsingBlockInputStream.h @@ -227,7 +227,7 @@ private: finished = true; { - std::unique_lock lock(mutex); + std::unique_lock lock(mutex); segmentator_condvar.notify_all(); reader_condvar.notify_all(); } @@ -255,4 +255,4 @@ private: void onBackgroundException(); }; -}; +} From 463567061048404c2e8a6fec07a19f9ba48e3692 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 5 Dec 2019 18:55:52 +0300 Subject: [PATCH 57/77] Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 755ca30793c..2094d6db33f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1005,7 +1005,7 @@ Possible values: Recommended values: - - 10000000 (100 times a second) nanosecods and more for for single queries. + - 10000000 (100 times a second) nanoseconds and less for single queries. - 1000000000 (once a second) for cluster-wide profiling. - 0 for turning off the timer. From 176dbfd54bce97962c94b03a613f36c06c3fed28 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 5 Dec 2019 18:56:12 +0300 Subject: [PATCH 58/77] Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 2094d6db33f..8e4321e0c33 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1001,7 +1001,7 @@ Sets the period for a real clock timer of the query profiler. Real clock timer c Possible values: -- Positive integer number of nanoseconds. +- Positive integer number, in nanoseconds. Recommended values: From 2216dfb45f44bb69eb4832546cee0c656b45d2bb Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 5 Dec 2019 18:56:31 +0300 Subject: [PATCH 59/77] Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 8e4321e0c33..e17d55ecdbf 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1012,7 +1012,7 @@ Possible values: Type: [UInt64](../../data_types/int_uint.md). -Default value: 1000000000 nanoseconds. +Default value: 1000000000 nanoseconds (once a second). **See Also** From dde164ab33848db0434b35cf1dcb3d51a2a1cdf6 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 5 Dec 2019 18:57:14 +0300 Subject: [PATCH 60/77] Update docs/en/operations/settings/settings.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index e17d55ecdbf..7c55731202e 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -1020,7 +1020,7 @@ Default value: 1000000000 nanoseconds (once a second). ## query_profiler_cpu_time_period_ns {#query_profiler_cpu_time_period_ns} -Sets the period for a CPU clock timer of the query profiler. Real clock timer counts CPU time. +Sets the period for a CPU clock timer of the query profiler. This timer counts only CPU time. Possible values: From 387cbca505c8afbe8d44a9b1da3377fdf3bbf5c5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 5 Dec 2019 19:36:51 +0300 Subject: [PATCH 61/77] Introduction refactoring + a bunch of docs fixes (#8010) * Create SECURITY.md * [experimental] auto-mark documentation PRs with labels * revert #6544 * Sync RPM packages instructions to other docs languages * Move tutorial to documentation with old content (for now) * refactor installation guide a bit * add ../en/getting_started/index.md * Rename ya_metrica_task.md * Rename ya_metrica_task.md * Refactor Yandex.Metrica dataset description * WIP on rewriting tutorial * tmp commit * lots of docs fixes * partially revert c136bee4ce9cfbec249aa1d729b5f88d34b90d2f * try to fix docs build in CI * try to fix docs build in CI * few minor improvements * Quick refactoring of last portion of tutorial (not thoroughly tested though) * fix link --- .../example_datasets/metrica.md | 91 +-- docs/en/getting_started/index.md | 145 +--- docs/en/getting_started/install.md | 153 +++++ docs/en/getting_started/tutorial.md | 645 +++++++++++++++++ .../{ya_metrika_task.md => history.md} | 4 +- docs/fa/getting_started/index.md | 198 +----- docs/fa/getting_started/install.md | 199 ++++++ docs/fa/getting_started/tutorial.md | 1 + .../{ya_metrika_task.md => history.md} | 2 +- docs/ja/changelog.md | 1 + docs/ja/data_types/array.md | 1 + docs/ja/data_types/boolean.md | 1 + docs/ja/data_types/date.md | 1 + docs/ja/data_types/datetime.md | 1 + docs/ja/data_types/decimal.md | 1 + docs/ja/data_types/domains/ipv4.md | 1 + docs/ja/data_types/domains/ipv6.md | 1 + docs/ja/data_types/domains/overview.md | 1 + docs/ja/data_types/enum.md | 1 + docs/ja/data_types/fixedstring.md | 1 + docs/ja/data_types/float.md | 1 + docs/ja/data_types/index.md | 1 + docs/ja/data_types/int_uint.md | 1 + .../aggregatefunction.md | 1 + .../nested_data_structures/index.md | 1 + .../nested_data_structures/nested.md | 1 + docs/ja/data_types/nullable.md | 1 + .../special_data_types/expression.md | 1 + .../ja/data_types/special_data_types/index.md | 1 + .../data_types/special_data_types/interval.md | 1 + .../data_types/special_data_types/nothing.md | 1 + docs/ja/data_types/special_data_types/set.md | 1 + docs/ja/data_types/string.md | 1 + docs/ja/data_types/tuple.md | 1 + docs/ja/data_types/uuid.md | 1 + docs/ja/database_engines/index.md | 1 + docs/ja/database_engines/lazy.md | 1 + docs/ja/database_engines/mysql.md | 1 + docs/ja/development/architecture.md | 1 + docs/ja/development/build.md | 1 + docs/ja/development/build_cross_arm.md | 1 + docs/ja/development/build_cross_osx.md | 1 + docs/ja/development/build_osx.md | 1 + docs/ja/development/contrib.md | 1 + docs/ja/development/developer_instruction.md | 1 + docs/ja/development/index.md | 1 + docs/ja/development/style.md | 1 + docs/ja/development/tests.md | 1 + docs/ja/faq/general.md | 1 + .../example_datasets/amplab_benchmark.md | 1 + .../example_datasets/criteo.md | 1 + .../example_datasets/metrica.md | 1 + .../example_datasets/nyc_taxi.md | 1 + .../example_datasets/ontime.md | 1 + .../example_datasets/star_schema.md | 1 + .../example_datasets/wikistat.md | 1 + docs/ja/getting_started/index.md | 1 + docs/ja/getting_started/install.md | 1 + docs/ja/getting_started/tutorial.md | 1 + docs/ja/guides/apply_catboost_model.md | 1 + docs/ja/guides/index.md | 1 + docs/ja/images/column_oriented.gif | Bin 0 -> 45485 bytes docs/ja/images/logo.svg | 12 + docs/ja/images/row_oriented.gif | Bin 0 -> 41571 bytes docs/ja/index.md | 143 +--- docs/ja/interfaces/cli.md | 1 + docs/ja/interfaces/cpp.md | 1 + docs/ja/interfaces/formats.md | 1 + docs/ja/interfaces/http.md | 1 + docs/ja/interfaces/index.md | 1 + docs/ja/interfaces/jdbc.md | 1 + docs/ja/interfaces/odbc.md | 1 + docs/ja/interfaces/tcp.md | 1 + .../third-party/client_libraries.md | 1 + docs/ja/interfaces/third-party/gui.md | 1 + .../ja/interfaces/third-party/integrations.md | 1 + docs/ja/interfaces/third-party/proxy.md | 1 + docs/ja/introduction/distinctive_features.md | 1 + .../features_considered_disadvantages.md | 1 + docs/ja/introduction/history.md | 1 + docs/ja/introduction/performance.md | 1 + docs/ja/operations/access_rights.md | 1 + docs/ja/operations/backup.md | 1 + docs/ja/operations/configuration_files.md | 1 + docs/ja/operations/index.md | 1 + docs/ja/operations/monitoring.md | 1 + docs/ja/operations/quotas.md | 1 + docs/ja/operations/requirements.md | 1 + docs/ja/operations/server_settings/index.md | 1 + .../ja/operations/server_settings/settings.md | 1 + .../settings/constraints_on_settings.md | 1 + docs/ja/operations/settings/index.md | 1 + .../settings/permissions_for_queries.md | 1 + .../operations/settings/query_complexity.md | 1 + docs/ja/operations/settings/settings.md | 1 + .../operations/settings/settings_profiles.md | 1 + docs/ja/operations/settings/settings_users.md | 1 + docs/ja/operations/system_tables.md | 1 + .../table_engines/aggregatingmergetree.md | 1 + docs/ja/operations/table_engines/buffer.md | 1 + .../table_engines/collapsingmergetree.md | 1 + .../table_engines/custom_partitioning_key.md | 1 + .../ja/operations/table_engines/dictionary.md | 1 + .../operations/table_engines/distributed.md | 1 + .../operations/table_engines/external_data.md | 1 + docs/ja/operations/table_engines/file.md | 1 + .../table_engines/graphitemergetree.md | 1 + docs/ja/operations/table_engines/hdfs.md | 1 + docs/ja/operations/table_engines/index.md | 1 + docs/ja/operations/table_engines/jdbc.md | 1 + docs/ja/operations/table_engines/join.md | 1 + docs/ja/operations/table_engines/kafka.md | 1 + docs/ja/operations/table_engines/log.md | 1 + .../ja/operations/table_engines/log_family.md | 1 + .../table_engines/materializedview.md | 1 + docs/ja/operations/table_engines/memory.md | 1 + docs/ja/operations/table_engines/merge.md | 1 + docs/ja/operations/table_engines/mergetree.md | 1 + docs/ja/operations/table_engines/mysql.md | 1 + docs/ja/operations/table_engines/null.md | 1 + docs/ja/operations/table_engines/odbc.md | 1 + .../table_engines/replacingmergetree.md | 1 + .../operations/table_engines/replication.md | 1 + docs/ja/operations/table_engines/set.md | 1 + docs/ja/operations/table_engines/stripelog.md | 1 + .../table_engines/summingmergetree.md | 1 + docs/ja/operations/table_engines/tinylog.md | 1 + docs/ja/operations/table_engines/url.md | 1 + .../versionedcollapsingmergetree.md | 1 + docs/ja/operations/table_engines/view.md | 1 + docs/ja/operations/tips.md | 1 + docs/ja/operations/troubleshooting.md | 1 + docs/ja/operations/update.md | 1 + docs/ja/operations/utils/clickhouse-copier.md | 1 + docs/ja/operations/utils/clickhouse-local.md | 1 + docs/ja/operations/utils/index.md | 1 + .../agg_functions/combinators.md | 1 + docs/ja/query_language/agg_functions/index.md | 1 + .../agg_functions/parametric_functions.md | 1 + .../query_language/agg_functions/reference.md | 1 + docs/ja/query_language/alter.md | 1 + docs/ja/query_language/create.md | 1 + .../ja/query_language/dicts/external_dicts.md | 1 + .../dicts/external_dicts_dict.md | 1 + .../dicts/external_dicts_dict_layout.md | 1 + .../dicts/external_dicts_dict_lifetime.md | 1 + .../dicts/external_dicts_dict_sources.md | 1 + .../dicts/external_dicts_dict_structure.md | 1 + docs/ja/query_language/dicts/index.md | 1 + .../ja/query_language/dicts/internal_dicts.md | 1 + .../functions/arithmetic_functions.md | 1 + .../functions/array_functions.md | 1 + .../ja/query_language/functions/array_join.md | 1 + .../query_language/functions/bit_functions.md | 1 + .../functions/bitmap_functions.md | 1 + .../functions/comparison_functions.md | 1 + .../functions/conditional_functions.md | 1 + .../functions/date_time_functions.md | 1 + .../functions/encoding_functions.md | 1 + .../functions/ext_dict_functions.md | 1 + .../functions/functions_for_nulls.md | 1 + docs/ja/query_language/functions/geo.md | 1 + .../functions/hash_functions.md | 1 + .../functions/higher_order_functions.md | 1 + .../query_language/functions/in_functions.md | 1 + docs/ja/query_language/functions/index.md | 1 + .../functions/ip_address_functions.md | 1 + .../functions/json_functions.md | 1 + .../functions/logical_functions.md | 1 + .../functions/machine_learning_functions.md | 1 + .../functions/math_functions.md | 1 + .../functions/other_functions.md | 1 + .../functions/random_functions.md | 1 + .../functions/rounding_functions.md | 1 + .../functions/splitting_merging_functions.md | 1 + .../functions/string_functions.md | 1 + .../functions/string_replace_functions.md | 1 + .../functions/string_search_functions.md | 1 + .../functions/type_conversion_functions.md | 1 + .../query_language/functions/url_functions.md | 1 + .../functions/uuid_functions.md | 1 + .../functions/ym_dict_functions.md | 1 + docs/ja/query_language/index.md | 1 + docs/ja/query_language/insert_into.md | 1 + docs/ja/query_language/misc.md | 1 + docs/ja/query_language/operators.md | 1 + docs/ja/query_language/select.md | 1 + docs/ja/query_language/show.md | 1 + docs/ja/query_language/syntax.md | 1 + docs/ja/query_language/system.md | 1 + .../ja/query_language/table_functions/file.md | 1 + .../ja/query_language/table_functions/hdfs.md | 1 + .../query_language/table_functions/index.md | 1 + .../query_language/table_functions/input.md | 1 + .../ja/query_language/table_functions/jdbc.md | 1 + .../query_language/table_functions/merge.md | 1 + .../query_language/table_functions/mysql.md | 1 + .../query_language/table_functions/numbers.md | 1 + .../ja/query_language/table_functions/odbc.md | 1 + .../query_language/table_functions/remote.md | 1 + docs/ja/query_language/table_functions/url.md | 1 + docs/ja/roadmap.md | 1 + docs/ja/security_changelog.md | 1 + docs/redirects.txt | 1 + docs/ru/getting_started/index.md | 136 +--- docs/ru/getting_started/install.md | 144 ++++ docs/ru/getting_started/tutorial.md | 1 + .../{ya_metrika_task.md => history.md} | 2 +- docs/toc_en.yml | 6 +- docs/toc_fa.yml | 16 +- docs/toc_ja.yml | 11 +- docs/toc_ru.yml | 8 +- docs/toc_zh.yml | 9 +- docs/tools/make_links.sh | 2 +- .../mkdocs-material-theme/assets/flags/ja.svg | 11 +- .../partials/language/ja.html | 6 + .../example_datasets/metrica.md | 1 + docs/zh/getting_started/index.md | 156 +---- docs/zh/getting_started/install.md | 152 ++++ docs/zh/getting_started/tutorial.md | 1 + .../{ya_metrika_task.md => history.md} | 2 +- website/nginx/default.conf | 2 + website/tutorial.html | 649 ------------------ 223 files changed, 1623 insertions(+), 1476 deletions(-) create mode 100644 docs/en/getting_started/install.md create mode 100644 docs/en/getting_started/tutorial.md rename docs/en/introduction/{ya_metrika_task.md => history.md} (98%) create mode 100644 docs/fa/getting_started/install.md create mode 120000 docs/fa/getting_started/tutorial.md rename docs/fa/introduction/{ya_metrika_task.md => history.md} (99%) create mode 120000 docs/ja/changelog.md create mode 120000 docs/ja/data_types/array.md create mode 120000 docs/ja/data_types/boolean.md create mode 120000 docs/ja/data_types/date.md create mode 120000 docs/ja/data_types/datetime.md create mode 120000 docs/ja/data_types/decimal.md create mode 120000 docs/ja/data_types/domains/ipv4.md create mode 120000 docs/ja/data_types/domains/ipv6.md create mode 120000 docs/ja/data_types/domains/overview.md create mode 120000 docs/ja/data_types/enum.md create mode 120000 docs/ja/data_types/fixedstring.md create mode 120000 docs/ja/data_types/float.md create mode 120000 docs/ja/data_types/index.md create mode 120000 docs/ja/data_types/int_uint.md create mode 120000 docs/ja/data_types/nested_data_structures/aggregatefunction.md create mode 120000 docs/ja/data_types/nested_data_structures/index.md create mode 120000 docs/ja/data_types/nested_data_structures/nested.md create mode 120000 docs/ja/data_types/nullable.md create mode 120000 docs/ja/data_types/special_data_types/expression.md create mode 120000 docs/ja/data_types/special_data_types/index.md create mode 120000 docs/ja/data_types/special_data_types/interval.md create mode 120000 docs/ja/data_types/special_data_types/nothing.md create mode 120000 docs/ja/data_types/special_data_types/set.md create mode 120000 docs/ja/data_types/string.md create mode 120000 docs/ja/data_types/tuple.md create mode 120000 docs/ja/data_types/uuid.md create mode 120000 docs/ja/database_engines/index.md create mode 120000 docs/ja/database_engines/lazy.md create mode 120000 docs/ja/database_engines/mysql.md create mode 120000 docs/ja/development/architecture.md create mode 120000 docs/ja/development/build.md create mode 120000 docs/ja/development/build_cross_arm.md create mode 120000 docs/ja/development/build_cross_osx.md create mode 120000 docs/ja/development/build_osx.md create mode 120000 docs/ja/development/contrib.md create mode 120000 docs/ja/development/developer_instruction.md create mode 120000 docs/ja/development/index.md create mode 120000 docs/ja/development/style.md create mode 120000 docs/ja/development/tests.md create mode 120000 docs/ja/faq/general.md create mode 120000 docs/ja/getting_started/example_datasets/amplab_benchmark.md create mode 120000 docs/ja/getting_started/example_datasets/criteo.md create mode 120000 docs/ja/getting_started/example_datasets/metrica.md create mode 120000 docs/ja/getting_started/example_datasets/nyc_taxi.md create mode 120000 docs/ja/getting_started/example_datasets/ontime.md create mode 120000 docs/ja/getting_started/example_datasets/star_schema.md create mode 120000 docs/ja/getting_started/example_datasets/wikistat.md create mode 120000 docs/ja/getting_started/index.md create mode 120000 docs/ja/getting_started/install.md create mode 120000 docs/ja/getting_started/tutorial.md create mode 120000 docs/ja/guides/apply_catboost_model.md create mode 120000 docs/ja/guides/index.md create mode 100644 docs/ja/images/column_oriented.gif create mode 100644 docs/ja/images/logo.svg create mode 100644 docs/ja/images/row_oriented.gif mode change 100644 => 120000 docs/ja/index.md create mode 120000 docs/ja/interfaces/cli.md create mode 120000 docs/ja/interfaces/cpp.md create mode 120000 docs/ja/interfaces/formats.md create mode 120000 docs/ja/interfaces/http.md create mode 120000 docs/ja/interfaces/index.md create mode 120000 docs/ja/interfaces/jdbc.md create mode 120000 docs/ja/interfaces/odbc.md create mode 120000 docs/ja/interfaces/tcp.md create mode 120000 docs/ja/interfaces/third-party/client_libraries.md create mode 120000 docs/ja/interfaces/third-party/gui.md create mode 120000 docs/ja/interfaces/third-party/integrations.md create mode 120000 docs/ja/interfaces/third-party/proxy.md create mode 120000 docs/ja/introduction/distinctive_features.md create mode 120000 docs/ja/introduction/features_considered_disadvantages.md create mode 120000 docs/ja/introduction/history.md create mode 120000 docs/ja/introduction/performance.md create mode 120000 docs/ja/operations/access_rights.md create mode 120000 docs/ja/operations/backup.md create mode 120000 docs/ja/operations/configuration_files.md create mode 120000 docs/ja/operations/index.md create mode 120000 docs/ja/operations/monitoring.md create mode 120000 docs/ja/operations/quotas.md create mode 120000 docs/ja/operations/requirements.md create mode 120000 docs/ja/operations/server_settings/index.md create mode 120000 docs/ja/operations/server_settings/settings.md create mode 120000 docs/ja/operations/settings/constraints_on_settings.md create mode 120000 docs/ja/operations/settings/index.md create mode 120000 docs/ja/operations/settings/permissions_for_queries.md create mode 120000 docs/ja/operations/settings/query_complexity.md create mode 120000 docs/ja/operations/settings/settings.md create mode 120000 docs/ja/operations/settings/settings_profiles.md create mode 120000 docs/ja/operations/settings/settings_users.md create mode 120000 docs/ja/operations/system_tables.md create mode 120000 docs/ja/operations/table_engines/aggregatingmergetree.md create mode 120000 docs/ja/operations/table_engines/buffer.md create mode 120000 docs/ja/operations/table_engines/collapsingmergetree.md create mode 120000 docs/ja/operations/table_engines/custom_partitioning_key.md create mode 120000 docs/ja/operations/table_engines/dictionary.md create mode 120000 docs/ja/operations/table_engines/distributed.md create mode 120000 docs/ja/operations/table_engines/external_data.md create mode 120000 docs/ja/operations/table_engines/file.md create mode 120000 docs/ja/operations/table_engines/graphitemergetree.md create mode 120000 docs/ja/operations/table_engines/hdfs.md create mode 120000 docs/ja/operations/table_engines/index.md create mode 120000 docs/ja/operations/table_engines/jdbc.md create mode 120000 docs/ja/operations/table_engines/join.md create mode 120000 docs/ja/operations/table_engines/kafka.md create mode 120000 docs/ja/operations/table_engines/log.md create mode 120000 docs/ja/operations/table_engines/log_family.md create mode 120000 docs/ja/operations/table_engines/materializedview.md create mode 120000 docs/ja/operations/table_engines/memory.md create mode 120000 docs/ja/operations/table_engines/merge.md create mode 120000 docs/ja/operations/table_engines/mergetree.md create mode 120000 docs/ja/operations/table_engines/mysql.md create mode 120000 docs/ja/operations/table_engines/null.md create mode 120000 docs/ja/operations/table_engines/odbc.md create mode 120000 docs/ja/operations/table_engines/replacingmergetree.md create mode 120000 docs/ja/operations/table_engines/replication.md create mode 120000 docs/ja/operations/table_engines/set.md create mode 120000 docs/ja/operations/table_engines/stripelog.md create mode 120000 docs/ja/operations/table_engines/summingmergetree.md create mode 120000 docs/ja/operations/table_engines/tinylog.md create mode 120000 docs/ja/operations/table_engines/url.md create mode 120000 docs/ja/operations/table_engines/versionedcollapsingmergetree.md create mode 120000 docs/ja/operations/table_engines/view.md create mode 120000 docs/ja/operations/tips.md create mode 120000 docs/ja/operations/troubleshooting.md create mode 120000 docs/ja/operations/update.md create mode 120000 docs/ja/operations/utils/clickhouse-copier.md create mode 120000 docs/ja/operations/utils/clickhouse-local.md create mode 120000 docs/ja/operations/utils/index.md create mode 120000 docs/ja/query_language/agg_functions/combinators.md create mode 120000 docs/ja/query_language/agg_functions/index.md create mode 120000 docs/ja/query_language/agg_functions/parametric_functions.md create mode 120000 docs/ja/query_language/agg_functions/reference.md create mode 120000 docs/ja/query_language/alter.md create mode 120000 docs/ja/query_language/create.md create mode 120000 docs/ja/query_language/dicts/external_dicts.md create mode 120000 docs/ja/query_language/dicts/external_dicts_dict.md create mode 120000 docs/ja/query_language/dicts/external_dicts_dict_layout.md create mode 120000 docs/ja/query_language/dicts/external_dicts_dict_lifetime.md create mode 120000 docs/ja/query_language/dicts/external_dicts_dict_sources.md create mode 120000 docs/ja/query_language/dicts/external_dicts_dict_structure.md create mode 120000 docs/ja/query_language/dicts/index.md create mode 120000 docs/ja/query_language/dicts/internal_dicts.md create mode 120000 docs/ja/query_language/functions/arithmetic_functions.md create mode 120000 docs/ja/query_language/functions/array_functions.md create mode 120000 docs/ja/query_language/functions/array_join.md create mode 120000 docs/ja/query_language/functions/bit_functions.md create mode 120000 docs/ja/query_language/functions/bitmap_functions.md create mode 120000 docs/ja/query_language/functions/comparison_functions.md create mode 120000 docs/ja/query_language/functions/conditional_functions.md create mode 120000 docs/ja/query_language/functions/date_time_functions.md create mode 120000 docs/ja/query_language/functions/encoding_functions.md create mode 120000 docs/ja/query_language/functions/ext_dict_functions.md create mode 120000 docs/ja/query_language/functions/functions_for_nulls.md create mode 120000 docs/ja/query_language/functions/geo.md create mode 120000 docs/ja/query_language/functions/hash_functions.md create mode 120000 docs/ja/query_language/functions/higher_order_functions.md create mode 120000 docs/ja/query_language/functions/in_functions.md create mode 120000 docs/ja/query_language/functions/index.md create mode 120000 docs/ja/query_language/functions/ip_address_functions.md create mode 120000 docs/ja/query_language/functions/json_functions.md create mode 120000 docs/ja/query_language/functions/logical_functions.md create mode 120000 docs/ja/query_language/functions/machine_learning_functions.md create mode 120000 docs/ja/query_language/functions/math_functions.md create mode 120000 docs/ja/query_language/functions/other_functions.md create mode 120000 docs/ja/query_language/functions/random_functions.md create mode 120000 docs/ja/query_language/functions/rounding_functions.md create mode 120000 docs/ja/query_language/functions/splitting_merging_functions.md create mode 120000 docs/ja/query_language/functions/string_functions.md create mode 120000 docs/ja/query_language/functions/string_replace_functions.md create mode 120000 docs/ja/query_language/functions/string_search_functions.md create mode 120000 docs/ja/query_language/functions/type_conversion_functions.md create mode 120000 docs/ja/query_language/functions/url_functions.md create mode 120000 docs/ja/query_language/functions/uuid_functions.md create mode 120000 docs/ja/query_language/functions/ym_dict_functions.md create mode 120000 docs/ja/query_language/index.md create mode 120000 docs/ja/query_language/insert_into.md create mode 120000 docs/ja/query_language/misc.md create mode 120000 docs/ja/query_language/operators.md create mode 120000 docs/ja/query_language/select.md create mode 120000 docs/ja/query_language/show.md create mode 120000 docs/ja/query_language/syntax.md create mode 120000 docs/ja/query_language/system.md create mode 120000 docs/ja/query_language/table_functions/file.md create mode 120000 docs/ja/query_language/table_functions/hdfs.md create mode 120000 docs/ja/query_language/table_functions/index.md create mode 120000 docs/ja/query_language/table_functions/input.md create mode 120000 docs/ja/query_language/table_functions/jdbc.md create mode 120000 docs/ja/query_language/table_functions/merge.md create mode 120000 docs/ja/query_language/table_functions/mysql.md create mode 120000 docs/ja/query_language/table_functions/numbers.md create mode 120000 docs/ja/query_language/table_functions/odbc.md create mode 120000 docs/ja/query_language/table_functions/remote.md create mode 120000 docs/ja/query_language/table_functions/url.md create mode 120000 docs/ja/roadmap.md create mode 120000 docs/ja/security_changelog.md create mode 100644 docs/ru/getting_started/install.md create mode 120000 docs/ru/getting_started/tutorial.md rename docs/ru/introduction/{ya_metrika_task.md => history.md} (99%) create mode 120000 docs/zh/getting_started/example_datasets/metrica.md create mode 100644 docs/zh/getting_started/install.md create mode 120000 docs/zh/getting_started/tutorial.md rename docs/zh/introduction/{ya_metrika_task.md => history.md} (99%) delete mode 100644 website/tutorial.html diff --git a/docs/en/getting_started/example_datasets/metrica.md b/docs/en/getting_started/example_datasets/metrica.md index 19947273338..d89fe54f4eb 100644 --- a/docs/en/getting_started/example_datasets/metrica.md +++ b/docs/en/getting_started/example_datasets/metrica.md @@ -1,51 +1,62 @@ # Anonymized Yandex.Metrica Data -Dataset consists of two tables containing anonymized data about hits (`hits_v1`) and visits (`visits_v1`) of Yandex.Metrica. Each of the tables can be downloaded as a compressed `tsv.xz` file or as prepared partitions. In addition to that, an extended version of the `hits` table containing 100 million rows is available as [TSV](https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_100m_obfuscated_v1.tsv.xz) and as [prepared partitions](https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_100m_obfuscated_v1.tar.xz). +Dataset consists of two tables containing anonymized data about hits (`hits_v1`) and visits (`visits_v1`) of Yandex.Metrica. You can read more about Yandex.Metrica in [ClickHouse history](../../introduction/history.md) section. + +The dataset consists of two tables, either of them can be downloaded as a compressed `tsv.xz` file or as prepared partitions. In addition to that, an extended version of the `hits` table containing 100 million rows is available as TSV at and as prepared partitions at . ## Obtaining Tables from Prepared Partitions -**Download and import hits:** -```bash -$ curl -O https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_v1.tar -$ tar xvf hits_v1.tar -C /var/lib/clickhouse # path to ClickHouse data directory -$ # check permissions on unpacked data, fix if required -$ sudo service clickhouse-server restart -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" + +Download and import hits table: + +``` bash +curl -O https://clickhouse-datasets.s3.yandex.net/hits/partitions/hits_v1.tar +tar xvf hits_v1.tar -C /var/lib/clickhouse # path to ClickHouse data directory +# check permissions on unpacked data, fix if required +sudo service clickhouse-server restart +clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` -**Download and import visits:** -```bash -$ curl -O https://clickhouse-datasets.s3.yandex.net/visits/partitions/visits_v1.tar -$ tar xvf visits_v1.tar -C /var/lib/clickhouse # path to ClickHouse data directory -$ # check permissions on unpacked data, fix if required -$ sudo service clickhouse-server restart -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" +Download and import visits: + +``` bash +curl -O https://clickhouse-datasets.s3.yandex.net/visits/partitions/visits_v1.tar +tar xvf visits_v1.tar -C /var/lib/clickhouse # path to ClickHouse data directory +# check permissions on unpacked data, fix if required +sudo service clickhouse-server restart +clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` -## Obtaining Tables from Compressed tsv-file -**Download and import hits from compressed tsv-file** -```bash -$ curl https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv -$ # now create table -$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -$ clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" -$ # import data -$ cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 -$ # optionally you can optimize table -$ clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL" -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" +## Obtaining Tables from Compressed TSV File + +Download and import hits from compressed TSV file: + +``` bash +curl https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv +# now create table +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" +clickhouse-client --query "CREATE TABLE datasets.hits_v1 ( WatchID UInt64, JavaEnable UInt8, Title String, GoodEvent Int16, EventTime DateTime, EventDate Date, CounterID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RegionID UInt32, UserID UInt64, CounterClass Int8, OS UInt8, UserAgent UInt8, URL String, Referer String, URLDomain String, RefererDomain String, Refresh UInt8, IsRobot UInt8, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), ResolutionWidth UInt16, ResolutionHeight UInt16, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, FlashMinor2 String, NetMajor UInt8, NetMinor UInt8, UserAgentMajor UInt16, UserAgentMinor FixedString(2), CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, MobilePhone UInt8, MobilePhoneModel String, Params String, IPNetworkID UInt32, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, IsArtifical UInt8, WindowClientWidth UInt16, WindowClientHeight UInt16, ClientTimeZone Int16, ClientEventTime DateTime, SilverlightVersion1 UInt8, SilverlightVersion2 UInt8, SilverlightVersion3 UInt32, SilverlightVersion4 UInt16, PageCharset String, CodeVersion UInt32, IsLink UInt8, IsDownload UInt8, IsNotBounce UInt8, FUniqID UInt64, HID UInt32, IsOldCounter UInt8, IsEvent UInt8, IsParameter UInt8, DontCountHits UInt8, WithHash UInt8, HitColor FixedString(1), UTCEventTime DateTime, Age UInt8, Sex UInt8, Income UInt8, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), RemoteIP UInt32, RemoteIP6 FixedString(16), WindowName Int32, OpenerName Int32, HistoryLength Int16, BrowserLanguage FixedString(2), BrowserCountry FixedString(2), SocialNetwork String, SocialAction String, HTTPError UInt16, SendTiming Int32, DNSTiming Int32, ConnectTiming Int32, ResponseStartTiming Int32, ResponseEndTiming Int32, FetchTiming Int32, RedirectTiming Int32, DOMInteractiveTiming Int32, DOMContentLoadedTiming Int32, DOMCompleteTiming Int32, LoadEventStartTiming Int32, LoadEventEndTiming Int32, NSToDOMContentLoadedTiming Int32, FirstPaintTiming Int32, RedirectCount Int8, SocialSourceNetworkID UInt8, SocialSourcePage String, ParamPrice Int64, ParamOrderID String, ParamCurrency FixedString(3), ParamCurrencyID UInt16, GoalsReached Array(UInt32), OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, RefererHash UInt64, URLHash UInt64, CLID UInt32, YCLID UInt64, ShareService String, ShareURL String, ShareTitle String, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), IslandID FixedString(16), RequestNum UInt32, RequestTry UInt8) ENGINE = MergeTree() PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS index_granularity = 8192" +# import data +cat hits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.hits_v1 FORMAT TSV" --max_insert_block_size=100000 +# optionally you can optimize table +clickhouse-client --query "OPTIMIZE TABLE datasets.hits_v1 FINAL" +clickhouse-client --query "SELECT COUNT(*) FROM datasets.hits_v1" ``` -**Download and import visits from compressed tsv-file** -```bash -$ curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv -$ # now create table -$ clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" -$ clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign)" -$ # import data -$ cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 -$ # optionally you can optimize table -$ clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL" -$ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" +Download and import visits from compressed tsv-file: + +``` bash +curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv +# now create table +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS datasets" +clickhouse-client --query "CREATE TABLE datasets.visits_v1 ( CounterID UInt32, StartDate Date, Sign Int8, IsNew UInt8, VisitID UInt64, UserID UInt64, StartTime DateTime, Duration UInt32, UTCStartTime DateTime, PageViews Int32, Hits Int32, IsBounce UInt8, Referer String, StartURL String, RefererDomain String, StartURLDomain String, EndURL String, LinkURL String, IsDownload UInt8, TraficSourceID Int8, SearchEngineID UInt16, SearchPhrase String, AdvEngineID UInt8, PlaceID Int32, RefererCategories Array(UInt16), URLCategories Array(UInt16), URLRegions Array(UInt32), RefererRegions Array(UInt32), IsYandex UInt8, GoalReachesDepth Int32, GoalReachesURL Int32, GoalReachesAny Int32, SocialSourceNetworkID UInt8, SocialSourcePage String, MobilePhoneModel String, ClientEventTime DateTime, RegionID UInt32, ClientIP UInt32, ClientIP6 FixedString(16), RemoteIP UInt32, RemoteIP6 FixedString(16), IPNetworkID UInt32, SilverlightVersion3 UInt32, CodeVersion UInt32, ResolutionWidth UInt16, ResolutionHeight UInt16, UserAgentMajor UInt16, UserAgentMinor UInt16, WindowClientWidth UInt16, WindowClientHeight UInt16, SilverlightVersion2 UInt8, SilverlightVersion4 UInt16, FlashVersion3 UInt16, FlashVersion4 UInt16, ClientTimeZone Int16, OS UInt8, UserAgent UInt8, ResolutionDepth UInt8, FlashMajor UInt8, FlashMinor UInt8, NetMajor UInt8, NetMinor UInt8, MobilePhone UInt8, SilverlightVersion1 UInt8, Age UInt8, Sex UInt8, Income UInt8, JavaEnable UInt8, CookieEnable UInt8, JavascriptEnable UInt8, IsMobile UInt8, BrowserLanguage UInt16, BrowserCountry UInt16, Interests UInt16, Robotness UInt8, GeneralInterests Array(UInt16), Params Array(String), Goals Nested(ID UInt32, Serial UInt32, EventTime DateTime, Price Int64, OrderID String, CurrencyID UInt32), WatchIDs Array(UInt64), ParamSumPrice Int64, ParamCurrency FixedString(3), ParamCurrencyID UInt16, ClickLogID UInt64, ClickEventID Int32, ClickGoodEvent Int32, ClickEventTime DateTime, ClickPriorityID Int32, ClickPhraseID Int32, ClickPageID Int32, ClickPlaceID Int32, ClickTypeID Int32, ClickResourceID Int32, ClickCost UInt32, ClickClientIP UInt32, ClickDomainID UInt32, ClickURL String, ClickAttempt UInt8, ClickOrderID UInt32, ClickBannerID UInt32, ClickMarketCategoryID UInt32, ClickMarketPP UInt32, ClickMarketCategoryName String, ClickMarketPPName String, ClickAWAPSCampaignName String, ClickPageName String, ClickTargetType UInt16, ClickTargetPhraseID UInt64, ClickContextType UInt8, ClickSelectType Int8, ClickOptions String, ClickGroupBannerID Int32, OpenstatServiceName String, OpenstatCampaignID String, OpenstatAdID String, OpenstatSourceID String, UTMSource String, UTMMedium String, UTMCampaign String, UTMContent String, UTMTerm String, FromTag String, HasGCLID UInt8, FirstVisit DateTime, PredLastVisit Date, LastVisit Date, TotalVisits UInt32, TraficSource Nested(ID Int8, SearchEngineID UInt16, AdvEngineID UInt8, PlaceID UInt16, SocialSourceNetworkID UInt8, Domain String, SearchPhrase String, SocialSourcePage String), Attendance FixedString(16), CLID UInt32, YCLID UInt64, NormalizedRefererHash UInt64, SearchPhraseHash UInt64, RefererDomainHash UInt64, NormalizedStartURLHash UInt64, StartURLDomainHash UInt64, NormalizedEndURLHash UInt64, TopLevelDomain UInt64, URLScheme UInt64, OpenstatServiceNameHash UInt64, OpenstatCampaignIDHash UInt64, OpenstatAdIDHash UInt64, OpenstatSourceIDHash UInt64, UTMSourceHash UInt64, UTMMediumHash UInt64, UTMCampaignHash UInt64, UTMContentHash UInt64, UTMTermHash UInt64, FromHash UInt64, WebVisorEnabled UInt8, WebVisorActivity UInt32, ParsedParams Nested(Key1 String, Key2 String, Key3 String, Key4 String, Key5 String, ValueDouble Float64), Market Nested(Type UInt8, GoalID UInt32, OrderID String, OrderPrice Int64, PP UInt32, DirectPlaceID UInt32, DirectOrderID UInt32, DirectBannerID UInt32, GoodID String, GoodName String, GoodQuantity Int32, GoodPrice Int64), IslandID FixedString(16)) ENGINE = CollapsingMergeTree(StartDate, intHash32(UserID), (CounterID, StartDate, intHash32(UserID), VisitID), 8192, Sign)" +# import data +cat visits_v1.tsv | clickhouse-client --query "INSERT INTO datasets.visits_v1 FORMAT TSV" --max_insert_block_size=100000 +# optionally you can optimize table +clickhouse-client --query "OPTIMIZE TABLE datasets.visits_v1 FINAL" +clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1" ``` -## Queries -Examples of queries to these tables (they are named `test.hits` and `test.visits`) can be found among [stateful tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) and in some [performance tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance) of ClickHouse. +## Example Queries + +[ClickHouse tutorial](../../getting_started/tutorial.md) is based on Yandex.Metrica dataset and the recommended way to get started with this dataset is to just go through tutorial. + +Additional examples of queries to these tables can be found among [stateful tests](https://github.com/yandex/ClickHouse/tree/master/dbms/tests/queries/1_stateful) of ClickHouse (they are named `test.hists` and `test.visits` there). diff --git a/docs/en/getting_started/index.md b/docs/en/getting_started/index.md index ed7335b748b..bfdcb0e108a 100644 --- a/docs/en/getting_started/index.md +++ b/docs/en/getting_started/index.md @@ -1,147 +1,8 @@ # Getting Started -## System Requirements - -ClickHouse can run on any Linux, FreeBSD or Mac OS X with x86\_64 CPU architecture. - -Though pre-built binaries are typically compiled to leverage SSE 4.2 instruction set, so unless otherwise stated usage of CPU that supports it becomes an additional system requirement. Here's the command to check if current CPU has support for SSE 4.2: - -``` bash -$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" -``` - -## Installation - -### From DEB Packages - -Yandex ClickHouse team recommends using official pre-compiled `deb` packages for Debian or Ubuntu. - -To install official packages add the Yandex repository in `/etc/apt/sources.list` or in a separate `/etc/apt/sources.list.d/clickhouse.list` file: - -```bash -$ deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ -``` - -If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). - -Then run these commands to actually install packages: - -```bash -$ sudo apt-get install dirmngr # optional -$ sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional -$ sudo apt-get update -$ sudo apt-get install clickhouse-client clickhouse-server -``` - -You can also download and install packages manually from here: . - -### From RPM Packages - -Yandex ClickHouse team recommends using official pre-compiled `rpm` packages for CentOS, RedHat and all other rpm-based Linux distributions. - -First you need to add the official repository: - -```bash -$ sudo yum install yum-utils -$ sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG -$ sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 -``` - -If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). - -Then run these commands to actually install packages: - -```bash -$ sudo yum install clickhouse-server clickhouse-client -``` - -You can also download and install packages manually from here: . - -### From Docker Image - -To run ClickHouse inside Docker follow the guide on [Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/). Those images use official `deb` packages inside. - -### From Sources - -To manually compile ClickHouse, follow the instructions for [Linux](../development/build.md) or [Mac OS X](../development/build_osx.md). - -You can compile packages and install them or use programs without installing packages. Also by building manually you can disable SSE 4.2 requirement or build for AArch64 CPUs. - -```text -Client: dbms/programs/clickhouse-client -Server: dbms/programs/clickhouse-server -``` - -You'll need to create a data and metadata folders and `chown` them for the desired user. Their paths can be changed in server config (src/dbms/programs/server/config.xml), by default they are: -```text -/opt/clickhouse/data/default/ -/opt/clickhouse/metadata/default/ -``` - -On Gentoo you can just use `emerge clickhouse` to install ClickHouse from sources. - -## Launch - -To start the server as a daemon, run: - -``` bash -$ sudo service clickhouse-server start -``` - -If you don't have `service` command, run as - -``` bash -$ sudo /etc/init.d/clickhouse-server start -``` - - -See the logs in the `/var/log/clickhouse-server/` directory. - -If the server doesn't start, check the configurations in the file `/etc/clickhouse-server/config.xml`. - -You can also manually launch the server from the console: - -``` bash -$ clickhouse-server --config-file=/etc/clickhouse-server/config.xml -``` - -In this case, the log will be printed to the console, which is convenient during development. -If the configuration file is in the current directory, you don't need to specify the `--config-file` parameter. By default, it uses `./config.xml`. - -ClickHouse supports 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`. -For more information, see the section ["Configuration Files"](../operations/configuration_files.md). - -After launching server, you can use the command-line client to connect to it: - -``` bash -$ clickhouse-client -``` - -By default it connects to `localhost:9000` on behalf of the user `default` without a password. It can also be used to connect to a remote server using `--host` argument. - -The terminal must use UTF-8 encoding. -For more information, see the section ["Command-line client"](../interfaces/cli.md). - -Example: -``` bash -$ ./clickhouse-client -ClickHouse client version 0.0.18749. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.18749. -``` -```sql -SELECT 1 -``` -```text -┌─1─┐ -│ 1 │ -└───┘ -``` - -**Congratulations, the system works!** - -To continue experimenting, you can download one of test data sets or go through [tutorial](https://clickhouse.yandex/tutorial.html). +If you are new to ClickHouse and want to get a hands-on feeling of it's performance, first of all you need to go through the [installation process](install.md). After that you can: +* [Go through detailed tutorial](tutorial.md) +* [Experiment with example datasets](example_datasets/ontime.md) [Original article](https://clickhouse.yandex/docs/en/getting_started/) diff --git a/docs/en/getting_started/install.md b/docs/en/getting_started/install.md new file mode 100644 index 00000000000..e47500fa22f --- /dev/null +++ b/docs/en/getting_started/install.md @@ -0,0 +1,153 @@ +# Installation + +## System Requirements + +ClickHouse can run on any Linux, FreeBSD or Mac OS X with x86\_64, AArch64 or PowerPC64LE CPU architecture. + +Official pre-built binaries are typically compiled for x86\_64 and leverage SSE 4.2 instruction set, so unless otherwise stated usage of CPU that supports it becomes an additional system requirement. Here's the command to check if current CPU has support for SSE 4.2: + +``` bash +$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" +``` + +To run ClickHouse on processors that do not support SSE 4.2 or have AArch64 or PowerPC64LE architecture, you should [build ClickHouse from sources](#from-sources) with proper configuration adjustments. + +## Available Installation Options + +### From DEB Packages + +It is recommended to use official pre-compiled `deb` packages for Debian or Ubuntu. + +To install official packages add the Yandex repository in `/etc/apt/sources.list` or in a separate `/etc/apt/sources.list.d/clickhouse.list` file: + +``` +deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ +``` + +If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). + +Then run these commands to actually install packages: + +```bash +sudo apt-get install dirmngr # optional +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional +sudo apt-get update +sudo apt-get install clickhouse-client clickhouse-server +``` + +You can also download and install packages manually from here: . + +### From RPM Packages + +It is recommended to use official pre-compiled `rpm` packages for CentOS, RedHat and all other rpm-based Linux distributions. + +First you need to add the official repository: + +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` + +If you want to use the most recent version, replace `stable` with `testing` (this is recommended for your testing environments). + +Then run these commands to actually install packages: + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +You can also download and install packages manually from here: . + +### From Docker Image + +To run ClickHouse inside Docker follow the guide on [Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/). Those images use official `deb` packages inside. + +### From Sources + +To manually compile ClickHouse, follow the instructions for [Linux](../development/build.md) or [Mac OS X](../development/build_osx.md). + +You can compile packages and install them or use programs without installing packages. Also by building manually you can disable SSE 4.2 requirement or build for AArch64 CPUs. + +``` +Client: dbms/programs/clickhouse-client +Server: dbms/programs/clickhouse-server +``` + +You'll need to create a data and metadata folders and `chown` them for the desired user. Their paths can be changed in server config (src/dbms/programs/server/config.xml), by default they are: +``` +/opt/clickhouse/data/default/ +/opt/clickhouse/metadata/default/ +``` + +On Gentoo you can just use `emerge clickhouse` to install ClickHouse from sources. + +## Launch + +To start the server as a daemon, run: + +``` bash +$ sudo service clickhouse-server start +``` + +If you don't have `service` command, run as + +``` bash +$ sudo /etc/init.d/clickhouse-server start +``` + + +See the logs in the `/var/log/clickhouse-server/` directory. + +If the server doesn't start, check the configurations in the file `/etc/clickhouse-server/config.xml`. + +You can also manually launch the server from the console: + +``` bash +$ clickhouse-server --config-file=/etc/clickhouse-server/config.xml +``` + +In this case, the log will be printed to the console, which is convenient during development. +If the configuration file is in the current directory, you don't need to specify the `--config-file` parameter. By default, it uses `./config.xml`. + +ClickHouse supports 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`. +For more information, see the section ["Configuration Files"](../operations/configuration_files.md). + +After launching server, you can use the command-line client to connect to it: + +``` bash +$ clickhouse-client +``` + +By default it connects to `localhost:9000` on behalf of the user `default` without a password. It can also be used to connect to a remote server using `--host` argument. + +The terminal must use UTF-8 encoding. +For more information, see the section ["Command-line client"](../interfaces/cli.md). + +Example: +``` bash +$ ./clickhouse-client +ClickHouse client version 0.0.18749. +Connecting to localhost:9000. +Connected to ClickHouse server version 0.0.18749. + +:) SELECT 1 + +SELECT 1 + +┌─1─┐ +│ 1 │ +└───┘ + +1 rows in set. Elapsed: 0.003 sec. + +:) +``` + +**Congratulations, the system works!** + +To continue experimenting, you can download one of test data sets or go through [tutorial](https://clickhouse.yandex/tutorial.html). + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/install/) diff --git a/docs/en/getting_started/tutorial.md b/docs/en/getting_started/tutorial.md new file mode 100644 index 00000000000..acdd9074beb --- /dev/null +++ b/docs/en/getting_started/tutorial.md @@ -0,0 +1,645 @@ +# ClickHouse Tutorial + +## What to Expect from This Tutorial? + +By going through this tutorial you'll learn how to set up basic ClickHouse cluster, it'll be small, but fault tolerant and scalable. We will use one of example datasets to fill it with data and execute some demo queries. + +## Single Node Setup + +To postpone complexities of distributed environment, we'll start with deploying ClickHouse on a single server or virtual machine. ClickHouse is usually installed from [deb](index.md#from-deb-packages) or [rpm](index.md#from-rpm-packages) packages, but there are [alternatives](index.md#from-docker-image) for the operating systems that do no support them. + +For example, you have chosen `deb` packages and executed: +``` bash +sudo apt-get install dirmngr +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 + +echo "deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" | sudo tee /etc/apt/sources.list.d/clickhouse.list +sudo apt-get update + +sudo apt-get install -y clickhouse-server clickhouse-client +``` + +What do we have in the packages that got installed: + +* `clickhouse-client` package contains [clickhouse-client](../interfaces/cli.md) application, interactive ClickHouse console client. +* `clickhouse-common` package contains a ClickHouse executable file. +* `clickhouse-server` package contains configuration files to run ClickHouse as a server. + +Server config files are located in `/etc/clickhouse-server/`. Before going further please notice the `` element in `config.xml`. Path determines the location for data storage, so it should be located on volume with large disk capacity, the default value is `/var/lib/clickhouse/`. If you want to adjust the configuration it's not really handy to directly edit `config.xml` file, considering it might get rewritten on future package updates. Recommended way to override the config elements is to create [files in config.d directory](../operations/configuration_files.md) which serve as "patches" to config.xml. + +As you might have noticed, `clickhouse-server` is not launched automatically after package installation. It won't be automatically restarted after updates either. The way you start the server depends on your init system, usually it's: + +``` bash +sudo service clickhouse-server start +``` +or + +``` bash +sudo /etc/init.d/clickhouse-server start +``` + +The default location for server logs is `/var/log/clickhouse-server/`. Server will be ready to handle client connections once `Ready for connections` message was logged. + +Once the `clickhouse-server` is up and running, we can use `clickhouse-client` to connect to the server and run some test queries like `SELECT "Hello, world!";`. + +
Quick tips for clickhouse-client +Interactive mode: +``` bash +clickhouse-client +clickhouse-client --host=... --port=... --user=... --password=... +``` + +Enable multiline queries: +``` bash +clickhouse-client -m +clickhouse-client --multiline +``` + +Run queries in batch-mode: +``` bash +clickhouse-client --query='SELECT 1' +echo 'SELECT 1' | clickhouse-client +clickhouse-client <<< 'SELECT 1' +``` + +Insert data from a file in specified format: +``` bash +clickhouse-client --query='INSERT INTO table VALUES' < data.txt +clickhouse-client --query='INSERT INTO table FORMAT TabSeparated' < data.tsv +``` +
+ +## Import Sample Dataset + +Now it's time to fill our ClickHouse server with some sample data. In this tutorial we'll use anonymized data of Yandex.Metrica, the first service that run ClickHouse in production way before it became open-source (more on that in [history section](../introduction/history.md)). There are [multiple ways to import Yandex.Metrica dataset](example_datasets/metrica.md) and for the sake of the tutorial we'll go with the most realistic one. + +### Download and Extract Table Data + +``` bash +curl https://clickhouse-datasets.s3.yandex.net/hits/tsv/hits_v1.tsv.xz | unxz --threads=`nproc` > hits_v1.tsv +curl https://clickhouse-datasets.s3.yandex.net/visits/tsv/visits_v1.tsv.xz | unxz --threads=`nproc` > visits_v1.tsv +``` + +The extracted files are about 10GB in size. + +### Create Tables + +Tables are logically grouped into "databases". There's a `default` database, but we'll create a new one named `tutorial`: + +``` bash +clickhouse-client --query "CREATE DATABASE IF NOT EXISTS tutorial" +``` + +Syntax for creating tables is way more complicated compared to databases (see [reference](../query_language/create.md). In general `CREATE TABLE` statement has to specify three key things: + +1. Name of table to create. +2. Table schema, i.e. list of columns and their [data types](../data_types/index.md). +3. [Table engine](../operations/table_engines/index.md) and it's settings, which determines all the details on how queries to this table will be physically executed. + +Yandex.Metrica is a web analytics service and sample dataset doesn't cover it's full functionality, so there are only two tables to create: + +* `hits` is a table with each action done by all users on all websites covered by the service. +* `visits` is a table that contains pre-built sessions instead of individual actions. + +Let's see and execute the real create table queries for these tables: + +``` sql +CREATE TABLE tutorial.hits_v1 +( + `WatchID` UInt64, + `JavaEnable` UInt8, + `Title` String, + `GoodEvent` Int16, + `EventTime` DateTime, + `EventDate` Date, + `CounterID` UInt32, + `ClientIP` UInt32, + `ClientIP6` FixedString(16), + `RegionID` UInt32, + `UserID` UInt64, + `CounterClass` Int8, + `OS` UInt8, + `UserAgent` UInt8, + `URL` String, + `Referer` String, + `URLDomain` String, + `RefererDomain` String, + `Refresh` UInt8, + `IsRobot` UInt8, + `RefererCategories` Array(UInt16), + `URLCategories` Array(UInt16), + `URLRegions` Array(UInt32), + `RefererRegions` Array(UInt32), + `ResolutionWidth` UInt16, + `ResolutionHeight` UInt16, + `ResolutionDepth` UInt8, + `FlashMajor` UInt8, + `FlashMinor` UInt8, + `FlashMinor2` String, + `NetMajor` UInt8, + `NetMinor` UInt8, + `UserAgentMajor` UInt16, + `UserAgentMinor` FixedString(2), + `CookieEnable` UInt8, + `JavascriptEnable` UInt8, + `IsMobile` UInt8, + `MobilePhone` UInt8, + `MobilePhoneModel` String, + `Params` String, + `IPNetworkID` UInt32, + `TraficSourceID` Int8, + `SearchEngineID` UInt16, + `SearchPhrase` String, + `AdvEngineID` UInt8, + `IsArtifical` UInt8, + `WindowClientWidth` UInt16, + `WindowClientHeight` UInt16, + `ClientTimeZone` Int16, + `ClientEventTime` DateTime, + `SilverlightVersion1` UInt8, + `SilverlightVersion2` UInt8, + `SilverlightVersion3` UInt32, + `SilverlightVersion4` UInt16, + `PageCharset` String, + `CodeVersion` UInt32, + `IsLink` UInt8, + `IsDownload` UInt8, + `IsNotBounce` UInt8, + `FUniqID` UInt64, + `HID` UInt32, + `IsOldCounter` UInt8, + `IsEvent` UInt8, + `IsParameter` UInt8, + `DontCountHits` UInt8, + `WithHash` UInt8, + `HitColor` FixedString(1), + `UTCEventTime` DateTime, + `Age` UInt8, + `Sex` UInt8, + `Income` UInt8, + `Interests` UInt16, + `Robotness` UInt8, + `GeneralInterests` Array(UInt16), + `RemoteIP` UInt32, + `RemoteIP6` FixedString(16), + `WindowName` Int32, + `OpenerName` Int32, + `HistoryLength` Int16, + `BrowserLanguage` FixedString(2), + `BrowserCountry` FixedString(2), + `SocialNetwork` String, + `SocialAction` String, + `HTTPError` UInt16, + `SendTiming` Int32, + `DNSTiming` Int32, + `ConnectTiming` Int32, + `ResponseStartTiming` Int32, + `ResponseEndTiming` Int32, + `FetchTiming` Int32, + `RedirectTiming` Int32, + `DOMInteractiveTiming` Int32, + `DOMContentLoadedTiming` Int32, + `DOMCompleteTiming` Int32, + `LoadEventStartTiming` Int32, + `LoadEventEndTiming` Int32, + `NSToDOMContentLoadedTiming` Int32, + `FirstPaintTiming` Int32, + `RedirectCount` Int8, + `SocialSourceNetworkID` UInt8, + `SocialSourcePage` String, + `ParamPrice` Int64, + `ParamOrderID` String, + `ParamCurrency` FixedString(3), + `ParamCurrencyID` UInt16, + `GoalsReached` Array(UInt32), + `OpenstatServiceName` String, + `OpenstatCampaignID` String, + `OpenstatAdID` String, + `OpenstatSourceID` String, + `UTMSource` String, + `UTMMedium` String, + `UTMCampaign` String, + `UTMContent` String, + `UTMTerm` String, + `FromTag` String, + `HasGCLID` UInt8, + `RefererHash` UInt64, + `URLHash` UInt64, + `CLID` UInt32, + `YCLID` UInt64, + `ShareService` String, + `ShareURL` String, + `ShareTitle` String, + `ParsedParams` Nested( + Key1 String, + Key2 String, + Key3 String, + Key4 String, + Key5 String, + ValueDouble Float64), + `IslandID` FixedString(16), + `RequestNum` UInt32, + `RequestTry` UInt8 +) +ENGINE = MergeTree() +PARTITION BY toYYYYMM(EventDate) +ORDER BY (CounterID, EventDate, intHash32(UserID)) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 +``` + +``` sql +CREATE TABLE tutorial.visits_v1 +( + `CounterID` UInt32, + `StartDate` Date, + `Sign` Int8, + `IsNew` UInt8, + `VisitID` UInt64, + `UserID` UInt64, + `StartTime` DateTime, + `Duration` UInt32, + `UTCStartTime` DateTime, + `PageViews` Int32, + `Hits` Int32, + `IsBounce` UInt8, + `Referer` String, + `StartURL` String, + `RefererDomain` String, + `StartURLDomain` String, + `EndURL` String, + `LinkURL` String, + `IsDownload` UInt8, + `TraficSourceID` Int8, + `SearchEngineID` UInt16, + `SearchPhrase` String, + `AdvEngineID` UInt8, + `PlaceID` Int32, + `RefererCategories` Array(UInt16), + `URLCategories` Array(UInt16), + `URLRegions` Array(UInt32), + `RefererRegions` Array(UInt32), + `IsYandex` UInt8, + `GoalReachesDepth` Int32, + `GoalReachesURL` Int32, + `GoalReachesAny` Int32, + `SocialSourceNetworkID` UInt8, + `SocialSourcePage` String, + `MobilePhoneModel` String, + `ClientEventTime` DateTime, + `RegionID` UInt32, + `ClientIP` UInt32, + `ClientIP6` FixedString(16), + `RemoteIP` UInt32, + `RemoteIP6` FixedString(16), + `IPNetworkID` UInt32, + `SilverlightVersion3` UInt32, + `CodeVersion` UInt32, + `ResolutionWidth` UInt16, + `ResolutionHeight` UInt16, + `UserAgentMajor` UInt16, + `UserAgentMinor` UInt16, + `WindowClientWidth` UInt16, + `WindowClientHeight` UInt16, + `SilverlightVersion2` UInt8, + `SilverlightVersion4` UInt16, + `FlashVersion3` UInt16, + `FlashVersion4` UInt16, + `ClientTimeZone` Int16, + `OS` UInt8, + `UserAgent` UInt8, + `ResolutionDepth` UInt8, + `FlashMajor` UInt8, + `FlashMinor` UInt8, + `NetMajor` UInt8, + `NetMinor` UInt8, + `MobilePhone` UInt8, + `SilverlightVersion1` UInt8, + `Age` UInt8, + `Sex` UInt8, + `Income` UInt8, + `JavaEnable` UInt8, + `CookieEnable` UInt8, + `JavascriptEnable` UInt8, + `IsMobile` UInt8, + `BrowserLanguage` UInt16, + `BrowserCountry` UInt16, + `Interests` UInt16, + `Robotness` UInt8, + `GeneralInterests` Array(UInt16), + `Params` Array(String), + `Goals` Nested( + ID UInt32, + Serial UInt32, + EventTime DateTime, + Price Int64, + OrderID String, + CurrencyID UInt32), + `WatchIDs` Array(UInt64), + `ParamSumPrice` Int64, + `ParamCurrency` FixedString(3), + `ParamCurrencyID` UInt16, + `ClickLogID` UInt64, + `ClickEventID` Int32, + `ClickGoodEvent` Int32, + `ClickEventTime` DateTime, + `ClickPriorityID` Int32, + `ClickPhraseID` Int32, + `ClickPageID` Int32, + `ClickPlaceID` Int32, + `ClickTypeID` Int32, + `ClickResourceID` Int32, + `ClickCost` UInt32, + `ClickClientIP` UInt32, + `ClickDomainID` UInt32, + `ClickURL` String, + `ClickAttempt` UInt8, + `ClickOrderID` UInt32, + `ClickBannerID` UInt32, + `ClickMarketCategoryID` UInt32, + `ClickMarketPP` UInt32, + `ClickMarketCategoryName` String, + `ClickMarketPPName` String, + `ClickAWAPSCampaignName` String, + `ClickPageName` String, + `ClickTargetType` UInt16, + `ClickTargetPhraseID` UInt64, + `ClickContextType` UInt8, + `ClickSelectType` Int8, + `ClickOptions` String, + `ClickGroupBannerID` Int32, + `OpenstatServiceName` String, + `OpenstatCampaignID` String, + `OpenstatAdID` String, + `OpenstatSourceID` String, + `UTMSource` String, + `UTMMedium` String, + `UTMCampaign` String, + `UTMContent` String, + `UTMTerm` String, + `FromTag` String, + `HasGCLID` UInt8, + `FirstVisit` DateTime, + `PredLastVisit` Date, + `LastVisit` Date, + `TotalVisits` UInt32, + `TraficSource` Nested( + ID Int8, + SearchEngineID UInt16, + AdvEngineID UInt8, + PlaceID UInt16, + SocialSourceNetworkID UInt8, + Domain String, + SearchPhrase String, + SocialSourcePage String), + `Attendance` FixedString(16), + `CLID` UInt32, + `YCLID` UInt64, + `NormalizedRefererHash` UInt64, + `SearchPhraseHash` UInt64, + `RefererDomainHash` UInt64, + `NormalizedStartURLHash` UInt64, + `StartURLDomainHash` UInt64, + `NormalizedEndURLHash` UInt64, + `TopLevelDomain` UInt64, + `URLScheme` UInt64, + `OpenstatServiceNameHash` UInt64, + `OpenstatCampaignIDHash` UInt64, + `OpenstatAdIDHash` UInt64, + `OpenstatSourceIDHash` UInt64, + `UTMSourceHash` UInt64, + `UTMMediumHash` UInt64, + `UTMCampaignHash` UInt64, + `UTMContentHash` UInt64, + `UTMTermHash` UInt64, + `FromHash` UInt64, + `WebVisorEnabled` UInt8, + `WebVisorActivity` UInt32, + `ParsedParams` Nested( + Key1 String, + Key2 String, + Key3 String, + Key4 String, + Key5 String, + ValueDouble Float64), + `Market` Nested( + Type UInt8, + GoalID UInt32, + OrderID String, + OrderPrice Int64, + PP UInt32, + DirectPlaceID UInt32, + DirectOrderID UInt32, + DirectBannerID UInt32, + GoodID String, + GoodName String, + GoodQuantity Int32, + GoodPrice Int64), + `IslandID` FixedString(16) +) +ENGINE = CollapsingMergeTree(Sign) +PARTITION BY toYYYYMM(StartDate) +ORDER BY (CounterID, StartDate, intHash32(UserID), VisitID) +SAMPLE BY intHash32(UserID) +SETTINGS index_granularity = 8192 +``` + +You can execute those queries using interactive mode of `clickhouse-client` (just launch it in terminal without specifying a query in advance) or try some [alternative interface](../interfaces/index.md) if you ant. + +As we can see, `hits_v1` uses the [basic MergeTree engine](../operations/table_engines/mergetree.md), while the `visits_v1` uses the [Collapsing](../operations/table_engines/collapsingmergetree.md) variant. + +### Import Data + +Data import to ClickHouse is done via [INSERT INTO](../query_language/insert_into.md) query like in many other SQL databases. However data is usually provided in one of the [supported formats](../interfaces/formats.md) instead of `VALUES` clause (which is also supported). + +The files we downloaded earlier are in tab-separated format, so here's how to import them via console client: + +``` bash +clickhouse-client --query "INSERT INTO tutorial.hits_v1 FORMAT TSV" --max_insert_block_size=100000 < hits_v1.tsv +clickhouse-client --query "INSERT INTO tutorial.visits_v1 FORMAT TSV" --max_insert_block_size=100000 < visits_v1.tsv +``` + +ClickHouse has a lot of [settings to tune](../operations/settings/index.md) and one way to specify them in console client is via arguments, as we can see with `--max_insert_block_size`. The easiest way to figure out what settings are available, what do they mean and what the defaults are is to query the `system.settings` table: + +``` sql +SELECT name, value, changed, description +FROM system.settings +WHERE name LIKE '%max_insert_b%' +FORMAT TSV + +max_insert_block_size 1048576 0 "The maximum block size for insertion, if we control the creation of blocks for insertion." +``` + +Optionally you can [OPTIMIZE](../query_language/misc/#misc_operations-optimize) the tables after import. Tables that are configured with MergeTree-family engine always do merges of data parts in background to optimize data storage (or at least check if it makes sense). These queries will just force table engine to do storage optimization right now instead of some time later: +``` bash +clickhouse-client --query "OPTIMIZE TABLE tutorial.hits_v1 FINAL" +clickhouse-client --query "OPTIMIZE TABLE tutorial.visits_v1 FINAL" +``` + +This is I/O and CPU intensive operation so if the table constantly receives new data it's better to leave it alone and let merges run in background. + +Now we can check that the tables are successfully imported: +``` bash +clickhouse-client --query "SELECT COUNT(*) FROM tutorial.hits_v1" +clickhouse-client --query "SELECT COUNT(*) FROM tutorial.visits_v1" +``` + +## Example Queries + +``` sql +SELECT + StartURL AS URL, + AVG(Duration) AS AvgDuration +FROM tutorial.visits_v1 +WHERE StartDate BETWEEN '2014-03-23' AND '2014-03-30' +GROUP BY URL +ORDER BY AvgDuration DESC +LIMIT 10 +``` + +``` sql +SELECT + sum(Sign) AS visits, + sumIf(Sign, has(Goals.ID, 1105530)) AS goal_visits, + (100. * goal_visits) / visits AS goal_percent +FROM tutorial.visits_v1 +WHERE (CounterID = 912887) AND (toYYYYMM(StartDate) = 201403) AND (domain(StartURL) = 'yandex.ru') +``` + +## Cluster Deployment + +ClickHouse cluster is a homogenous cluster. Steps to set up: + +1. Install ClickHouse server on all machines of the cluster +2. Set up cluster configs in configuration files +3. Create local tables on each instance +4. Create a [Distributed table](../operations/table_engines/distributed.md) + +[Distributed table](../operations/table_engines/distributed.md) is actually a kind of "view" to local tables of ClickHouse cluster. SELECT query from a distributed table will be executed using resources of all cluster's shards. You may specify configs for multiple clusters and create multiple distributed tables providing views to different clusters. + +Example config for cluster with three shards, one replica each: +``` xml + + + + + example-perftest01j.yandex.ru + 9000 + + + + + example-perftest02j.yandex.ru + 9000 + + + + + example-perftest03j.yandex.ru + 9000 + + + + +``` + +For further demonstration let's create new local table with exactly the same `CREATE TABLE` query that we used for `hits_v1`, but different table name: +``` sql +CREATE TABLE tutorial.hits_local (...) ENGINE = MergeTree() ... +``` + +Creating a distributed table providing a view into local tables of the cluster: +``` sql +CREATE TABLE tutorial.hits_all AS tutorial.hits_local +ENGINE = Distributed(perftest_3shards_1replicas, tutorial, hits_local, rand()); +``` + +Common practice is to create similar Distributed tables on all machines of the cluster. This would allow to run distributed queries on any machine of the cluster. Also there's an alternative option to create temporary distributed table for a given SELECT query using [remote](../query_language/table_functions/remote.md) table function. + +Let's run [INSERT SELECT](../query_language/insert_into.md) into Distributed table to spread the table to multiple servers. + +``` sql +INSERT INTO tutorial.hits_all SELECT * FROM tutorial.hits_v1; +``` + +!!! warning "Notice" + This approach is not suitable for sharding of large tables. There's a separate tool [clickhouse-copier](../operations/utils/clickhouse-copier.md) that can re-shard arbitrary large tables. + +As you could expect computationally heavy queries are executed N times faster being launched on 3 servers instead of one. + +In this case we have used a cluster with 3 shards each contains a single replica. + +To provide resilience in production environment we recommend that each shard should contain 2-3 replicas distributed between multiple data-centers. Note that ClickHouse supports unlimited number of replicas. + +Example config for cluster of one shard containing three replicas: +``` xml + + ... + + + + example-perftest01j.yandex.ru + 9000 + + + example-perftest02j.yandex.ru + 9000 + + + example-perftest03j.yandex.ru + 9000 + + + + +``` + +To enable native replication ZooKeeper is required. ClickHouse will take care of data consistency on all replicas and run restore procedure after failure + automatically. It's recommended to deploy ZooKeeper cluster to separate servers. + +ZooKeeper is not a strict requirement: in some simple cases you can duplicate the data by writing it into all the replicas from your application code. This approach is **not** recommended, in this case ClickHouse won't be able to + guarantee data consistency on all replicas. This remains the responsibility of your application. + +ZooKeeper locations need to be specified in configuration file: +``` xml + + + zoo01.yandex.ru + 2181 + + + zoo02.yandex.ru + 2181 + + + zoo03.yandex.ru + 2181 + + +``` + +Also we need to set macros for identifying each shard and replica, it will be used on table creation: +``` xml + + 01 + 01 + +``` + +If there are no replicas at the moment on replicated table creation, a new first replica will be instantiated. If there are already live replicas, new replica will clone the data from existing ones. You have an option to create all replicated tables first and that insert data to it. Another option is to create some replicas and add the others after or during data insertion. + +``` sql +CREATE TABLE tutorial.hits_replica (...) +ENGINE = ReplcatedMergeTree( + '/clickhouse_perftest/tables/{shard}/hits', + '{replica}' +) +... +``` + +Here we use [ReplicatedMergeTree](../operations/table_engines/replication.md) table engine. In parameters we specify ZooKeeper path containing shard and replica identifiers. + +``` sql +INSERT INTO tutorial.hits_replica SELECT * FROM tutorial.hits_local; +``` +Replication operates in multi-master mode. Data can be loaded into any replica and it will be synced with other instances automatically. Replication is asynchronous so at a given moment of time not all replicas may contain recently inserted data. To allow data insertion at least one replica should be up. Others will sync up data and repair consistency once they will become active again. Please notice that such approach allows for the low possibility of loss of just appended data. diff --git a/docs/en/introduction/ya_metrika_task.md b/docs/en/introduction/history.md similarity index 98% rename from docs/en/introduction/ya_metrika_task.md rename to docs/en/introduction/history.md index 41b33eff581..e8f373880f1 100644 --- a/docs/en/introduction/ya_metrika_task.md +++ b/docs/en/introduction/history.md @@ -1,4 +1,4 @@ -# Yandex.Metrica Use Case +# ClickHouse History ClickHouse was originally developed to power [Yandex.Metrica](https://metrica.yandex.com/), [the second largest web analytics platform in the world](http://w3techs.com/technologies/overview/traffic_analysis/all), and continues to be the core component of this system. With more than 13 trillion records in the database and more than 20 billion events daily, ClickHouse allows generating custom reports on the fly directly from non-aggregated data. This article briefly covers the goals of ClickHouse in the early stages of its development. @@ -47,4 +47,4 @@ OLAPServer worked well for non-aggregated data, but it had many restrictions tha To remove the limitations of OLAPServer and solve the problem of working with non-aggregated data for all reports, we developed the ClickHouse DBMS. -[Original article](https://clickhouse.yandex/docs/en/introduction/ya_metrika_task/) +[Original article](https://clickhouse.yandex/docs/en/introduction/history/) diff --git a/docs/fa/getting_started/index.md b/docs/fa/getting_started/index.md index 778393aed91..57496c474e2 100644 --- a/docs/fa/getting_started/index.md +++ b/docs/fa/getting_started/index.md @@ -1,197 +1,11 @@
+# ﻥﺪﺷ ﻉﻭﺮﺷ -# شروع به کار +ﻖﯾﺮﻃ ﺯﺍ ﺪﯾﺎﺑ ﻪﻤﻫ ﺯﺍ ﻝﻭﺍ ، ﺪﯿﻨﮐ ﺱﺎﺴﺣﺍ ﺍﺭ ﻥﺁ ﺩﺮﮑﻠﻤﻋ ﺪﯿﻫﺍﻮﺧ ﯽﻣ ﻭ ﺪﯿﺘﺴﻫ ﺩﺭﺍﻭ ﻩﺯﺎﺗ[ﺐﺼﻧ ﻞﺣﺍﺮﻣ](install.md). +ﺪﯿﻨﮐ ﺏﺎﺨﺘﻧﺍ ﺍﺭ ﺮﯾﺯ ﯼﺎﻫ ﻪﻨﯾﺰﮔ ﺯﺍ ﯽﮑﯾ ﺪﯿﻧﺍﻮﺗ ﯽﻣ ﻥﺁ ﺯﺍ ﺲﭘ: -## نیازمندی های سیستم - -این یک سیستم چند سکویی (Cross-Platform) نمی باشد. این ابزار نیاز به Linux Ubuntu Precise (12.04) یا جدیدتر، با معماری x86\_64 و پشتیبانی از SSE 4.2 می باشد. برای چک کردن SSE 4.2 خروجی دستور زیر را بررسی کنید: +* [ﺪﯿﻨﮐ ﯽﻃ ﺍﺭ ﻞﺼﻔﻣ ﺵﺯﻮﻣﺁ](tutorial.md) +* [ﺪﯿﻨﮐ ﺶﯾﺎﻣﺯﺁ ﻪﻧﻮﻤﻧ ﯼﺎﻫ ﻩﺩﺍﺩ ﺎﺑ](example_datasets/ontime.md) +[ﯽﻠﺻﺍ ﻪﻟﺎﻘﻣ](https://clickhouse.yandex/docs/fa/getting_started/)
- -```bash -grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" -``` - -
- -پیشنهاد می کنیم از Ubuntu TrustyT، Ubuntu Xenial یا Ubuntu Precise استفاده کنید. ترمینال باید از UTF-8 پشتیبانی کند. (به صورت پیش فرض در Ubuntu پشتیبانی می شود). - -## نصب - -### نصب از طریق پکیج های Debian/Ubuntu - -در فایل `/etc/apt/sources.list` (یا در یک فایل جدا `/etc/apt/sources.list.d/clickhouse.list`)، Repo زیر را اضافه کنید: - -
- -``` -deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ -``` - -
- -اگر شما میخوایید جدیدترین نسخه ی تست را استفاده کنید، 'stable' رو به 'testing' تغییر بدید. - -سپس دستورات زیر را اجرا کنید: - -
- -```bash -sudo apt-get install dirmngr # optional -sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional -sudo apt-get update -sudo apt-get install clickhouse-client clickhouse-server -``` - -
- -شما همچنین می توانید از طریق لینک زیر پکیج ClickHouse را به صورت دستی دانلود و نصب کنید: . - -ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید. - - RPM ﯼﺎﻫ ﻪﺘﺴﺑ ﺯﺍ ### - -.ﺪﻨﮐ ﯽﻣ ﻪﯿﺻﻮﺗ ﺲﮐﻮﻨﯿﻟ ﺮﺑ ﯽﻨﺘﺒﻣ rpm ﺮﺑ ﯽﻨﺘﺒﻣ ﯼﺎﻫ ﻊﯾﺯﻮﺗ ﺮﯾﺎﺳ ﻭ CentOS ، RedHat ﯼﺍ - - :ﺪﯿﻨﮐ ﻪﻓﺎﺿﺍ ﺍﺭ ﯽﻤﺳﺭ ﻥﺰﺨﻣ ﺪﯾﺎﺑ ﺍﺪﺘﺑﺍ - -```bash -sudo yum install yum-utils -sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG -sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 -``` - -.(ﺩﻮﺷ ﯽﻣ ﻪﯿﺻﻮﺗ ﺎﻤﺷ ﺶﯾﺎﻣﺯﺁ ﯼﺎﻫ ﻂﯿﺤﻣ ﯼﺍﺮﺑ ﻦﯾﺍ) ﺪﯿﻨﮐ ﻦﯾﺰﮕﯾﺎﺟ "ﺖﺴﺗ" ﺎﺑ ﺍﺭ "ﺭﺍﺪﯾﺎﭘ" - - :ﺪﯿﻨﮐ ﺐﺼﻧ ﺍﺭ ﺎﻫ ﻪﺘﺴﺑ ﻊﻗﺍﻭ ﺭﺩ ﺎﺗ ﺪﯿﻨﮐ ﺍﺮﺟﺍ ﺍﺭ ﺕﺍﺭﻮﺘﺳﺩ ﻦﯾﺍ ﺲﭙﺳ - -```bash -sudo yum install clickhouse-server clickhouse-client -``` - -. :ﺪﯿﻨﮐ ﺐﺼﻧ ﻭ ﯼﺮﯿﮔﺭﺎﺑ ﺎﺠﻨ - - Docker Image ﺯﺍ ### - -.ﺪﻨﻨﮐ ﯽﻣ ﻩﺩﺎﻔﺘﺳﺍ ﻞﺧﺍﺩ ﺭﺩ "deb" ﯽﻤﺳﺭ ﯼﺎﻫ ﻪﺘﺴﺑ ﺯﺍ ﺮﯾﻭﺎﺼﺗ ﻦﯾﺍ .ﺪﯿﻨﮐ ﻝﺎﺒﻧﺩ ﺍﺭ (/ht - - -### نصب از طریق Source - -برای Compile، دستورالعمل های فایل build.md را دنبال کنید: - -شما میتوانید پکیج را compile و نصب کنید. شما همچنین می توانید بدون نصب پکیج از برنامه ها استفاده کنید. - -
- -``` -Client: dbms/programs/clickhouse-client -Server: dbms/programs/clickhouse-server -``` - -
- -برای سرور، یک کاتالوگ با دیتا بسازید، مانند - -
- -``` -/opt/clickhouse/data/default/ -/opt/clickhouse/metadata/default/ -``` - -
- -(قابل تنظیم در تنظیمات سرور). 'chown' را برای کاربر دلخواه اجرا کنید. - -به مسیر لاگ ها در تنظیمات سرور توجه کنید (src/dbms/programs/config.xml). - -### روش های دیگر نصب - -Docker image: - -پکیج RPM برای CentOS یا RHEL: - -Gentoo: `emerge clickhouse` - -## راه اندازی - -برای استارت سرور (به صورت daemon)، دستور زیر را اجرا کنید: - -
- -```bash -sudo service clickhouse-server start -``` - -
- -لاگ های دایرکتوری `/var/log/clickhouse-server/` directory. را مشاهده کنید. - -اگر سرور استارت نشد، فایل تنظیمات را بررسی کنید `/etc/clickhouse-server/config.xml.` - -شما همچنین می توانید سرور را از طریق کنسول راه اندازی کنید: - -
- -```bash -clickhouse-server --config-file=/etc/clickhouse-server/config.xml -``` - -
- -در این مورد که مناسب زمان توسعه می باشد، لاگ ها در کنسول پرینت می شوند. اگر فایل تنظیمات در دایرکتوری جاری باشد، نیازی به مشخص کردن '--config-file' نمی باشد. به صورت پیش فرض از './config.xml' استفاده می شود. - -شما می توانید از کلاینت command-line برای اتصال به سرور استفاده کنید: - -
- -```bash -clickhouse-client -``` - -
- -پارامترهای پیش فرض، نشان از اتصال به localhost:9000 از طرف کاربر 'default' بدون پسورد را می دهد. از کلاینت میتوان برای اتصال به یک سرور remote استفاده کرد. مثال: - -
- -```bash -clickhouse-client --host=example.com -``` - -
- -برای اطلاعات بیشتر، بخش "کلاینت Command-line" را مشاهده کنید. - -چک کردن سیستم: - -
- -```bash -milovidov@hostname:~/work/metrica/src/dbms/src/Client$ ./clickhouse-client -ClickHouse client version 0.0.18749. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.18749. - -:) SELECT 1 - -SELECT 1 - -┌─1─┐ -│ 1 │ -└───┘ - -1 rows in set. Elapsed: 0.003 sec. - -:) -``` - -
- -**تبریک میگم، سیستم کار می کنه!** - -برای ادامه آزمایشات، شما میتوانید دیتاست های تستی را دریافت و امتحان کنید. - -
-[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/) diff --git a/docs/fa/getting_started/install.md b/docs/fa/getting_started/install.md new file mode 100644 index 00000000000..790c9381007 --- /dev/null +++ b/docs/fa/getting_started/install.md @@ -0,0 +1,199 @@ +
+ +# ﯼﺯﺍﺪﻧﺍ ﻩﺍﺭ ﻭ ﺐﺼﻧ + +## نیازمندی های سیستم + +ClickHouse ﺲﮐﻮﻨﯿﻟ ﻉﻮﻧ ﺮﻫ ﯼﻭﺭ ﺮﺑ ﺪﻧﺍﻮﺗ ﯽﻣ ، FreeBSD ﺎﯾ Mac OS X ﯼﺭﺎﻤﻌﻣ ﺎﺑ CPU x + +:ﺖﺳﺍ ﻩﺪﻣﺁ ، ﺪﻨﮐ ﯽﻣ ﯽﻧﺎﺒﯿﺘﺸﭘ SSE 4.2 ﺯﺍ ﯽﻠﻌﻓ CPU ﺎﯾﺁ ﻪﮑﻨﯾﺍ ﯽﺳﺭﺮﺑ ﯼﺍﺮﺑ ﺭﻮﺘﺳﺩ ﻦﯾﺍ + +
+ +```bash +grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" +``` + +
+ +ﺪﯾﺎﺑ ، ﺪﻧﺭﺍﺪﻧ PowerPC64LE ﺎﯾ AArch64 ﯼﺭﺎﻤﻌﻣ ﺎﯾ ﺪﻨﻨﮐ ﯽﻤﻧ ﯽﻧﺎﺒﯿﺘﺸﭘ SSE 4.2 ﺯﺍ ﻪﮐ[ClickHouse ﺪﯿﻨﮐ ﺩﺎﺠﯾﺍ ﻊﺑﺎﻨﻣ ﺯﺍ ﺍﺭ](#from-sources) ﺐﺳﺎﻨﻣ ﺕﺎﻤﯿﻈﻨﺗ ﺎﺑ + +##ﺩﻮﺟﻮﻣ ﺐﺼﻧ ﯼﺎﻫ ﻪﻨﯾﺰﮔ + +### نصب از طریق پکیج های Debian/Ubuntu {#from-deb-packages} + +در فایل `/etc/apt/sources.list` (یا در یک فایل جدا `/etc/apt/sources.list.d/clickhouse.list`)، Repo زیر را اضافه کنید: + +
+ +``` +deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ +``` + +
+ +اگر شما میخوایید جدیدترین نسخه ی تست را استفاده کنید، 'stable' رو به 'testing' تغییر بدید. + +سپس دستورات زیر را اجرا کنید: + +
+ +```bash +sudo apt-get install dirmngr # optional +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional +sudo apt-get update +sudo apt-get install clickhouse-client clickhouse-server +``` + +
+ +شما همچنین می توانید از طریق لینک زیر پکیج ClickHouse را به صورت دستی دانلود و نصب کنید: . + +ClickHouse دارای تنظیمات محدودیت دسترسی می باشد. این تنظیمات در فایل 'users.xml' (کنار 'config.xml') می باشد. به صورت پیش فرض دسترسی برای کاربر 'default' از همه جا بدون نیاز به پسورد وجود دارد. 'user/default/networks' را مشاهده کنید. برای اطلاعات بیشتر قسمت "تنظیمات فایل ها" را مشاهده کنید. + +### RPM ﯼﺎﻫ ﻪﺘﺴﺑ ﺯﺍ {#from-rpm-packages} + +.ﺪﻨﮐ ﯽﻣ ﻪﯿﺻﻮﺗ ﺲﮐﻮﻨﯿﻟ ﺮﺑ ﯽﻨﺘﺒﻣ rpm ﺮﺑ ﯽﻨﺘﺒﻣ ﯼﺎﻫ ﻊﯾﺯﻮﺗ ﺮﯾﺎﺳ ﻭ CentOS ، RedHat ﯼﺍ + + :ﺪﯿﻨﮐ ﻪﻓﺎﺿﺍ ﺍﺭ ﯽﻤﺳﺭ ﻥﺰﺨﻣ ﺪﯾﺎﺑ ﺍﺪﺘﺑﺍ + +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` + +.(ﺩﻮﺷ ﯽﻣ ﻪﯿﺻﻮﺗ ﺎﻤﺷ ﺶﯾﺎﻣﺯﺁ ﯼﺎﻫ ﻂﯿﺤﻣ ﯼﺍﺮﺑ ﻦﯾﺍ) ﺪﯿﻨﮐ ﻦﯾﺰﮕﯾﺎﺟ "ﺖﺴﺗ" ﺎﺑ ﺍﺭ "ﺭﺍﺪﯾﺎﭘ" + + :ﺪﯿﻨﮐ ﺐﺼﻧ ﺍﺭ ﺎﻫ ﻪﺘﺴﺑ ﻊﻗﺍﻭ ﺭﺩ ﺎﺗ ﺪﯿﻨﮐ ﺍﺮﺟﺍ ﺍﺭ ﺕﺍﺭﻮﺘﺳﺩ ﻦﯾﺍ ﺲﭙﺳ + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +. :ﺪﯿﻨﮐ ﺐﺼﻧ ﻭ ﯼﺮﯿﮔﺭﺎﺑ ﺎﺠﻨ + + Docker Image ﺯﺍ ### + +.ﺪﻨﻨﮐ ﯽﻣ ﻩﺩﺎﻔﺘﺳﺍ ﻞﺧﺍﺩ ﺭﺩ "deb" ﯽﻤﺳﺭ ﯼﺎﻫ ﻪﺘﺴﺑ ﺯﺍ ﺮﯾﻭﺎﺼﺗ ﻦﯾﺍ .ﺪﯿﻨﮐ ﻝﺎﺒﻧﺩ ﺍﺭ (/ht + + +### نصب از طریق Source {#from-sources} + +برای Compile، دستورالعمل های فایل build.md را دنبال کنید: + +شما میتوانید پکیج را compile و نصب کنید. شما همچنین می توانید بدون نصب پکیج از برنامه ها استفاده کنید. + +
+ +``` +Client: dbms/programs/clickhouse-client +Server: dbms/programs/clickhouse-server +``` + +
+ +برای سرور، یک کاتالوگ با دیتا بسازید، مانند + +
+ +``` +/opt/clickhouse/data/default/ +/opt/clickhouse/metadata/default/ +``` + +
+ +(قابل تنظیم در تنظیمات سرور). 'chown' را برای کاربر دلخواه اجرا کنید. + +به مسیر لاگ ها در تنظیمات سرور توجه کنید (src/dbms/programs/config.xml). + +### روش های دیگر نصب {#from-docker-image} + +Docker image: + +پکیج RPM برای CentOS یا RHEL: + +Gentoo: `emerge clickhouse` + +## راه اندازی + +برای استارت سرور (به صورت daemon)، دستور زیر را اجرا کنید: + +
+ +```bash +sudo service clickhouse-server start +``` + +
+ +لاگ های دایرکتوری `/var/log/clickhouse-server/` directory. را مشاهده کنید. + +اگر سرور استارت نشد، فایل تنظیمات را بررسی کنید `/etc/clickhouse-server/config.xml.` + +شما همچنین می توانید سرور را از طریق کنسول راه اندازی کنید: + +
+ +```bash +clickhouse-server --config-file=/etc/clickhouse-server/config.xml +``` + +
+ +در این مورد که مناسب زمان توسعه می باشد، لاگ ها در کنسول پرینت می شوند. اگر فایل تنظیمات در دایرکتوری جاری باشد، نیازی به مشخص کردن '--config-file' نمی باشد. به صورت پیش فرض از './config.xml' استفاده می شود. + +شما می توانید از کلاینت command-line برای اتصال به سرور استفاده کنید: + +
+ +```bash +clickhouse-client +``` + +
+ +پارامترهای پیش فرض، نشان از اتصال به localhost:9000 از طرف کاربر 'default' بدون پسورد را می دهد. از کلاینت میتوان برای اتصال به یک سرور remote استفاده کرد. مثال: + +
+ +```bash +clickhouse-client --host=example.com +``` + +
+ +برای اطلاعات بیشتر، بخش "کلاینت Command-line" را مشاهده کنید. + +چک کردن سیستم: + +
+ +```bash +milovidov@hostname:~/work/metrica/src/dbms/src/Client$ ./clickhouse-client +ClickHouse client version 0.0.18749. +Connecting to localhost:9000. +Connected to ClickHouse server version 0.0.18749. + +:) SELECT 1 + +SELECT 1 + +┌─1─┐ +│ 1 │ +└───┘ + +1 rows in set. Elapsed: 0.003 sec. + +:) +``` + +
+ +**تبریک میگم، سیستم کار می کنه!** + +برای ادامه آزمایشات، شما میتوانید دیتاست های تستی را دریافت و امتحان کنید. + +
+[مقاله اصلی](https://clickhouse.yandex/docs/fa/getting_started/install/) diff --git a/docs/fa/getting_started/tutorial.md b/docs/fa/getting_started/tutorial.md new file mode 120000 index 00000000000..8bc40816ab2 --- /dev/null +++ b/docs/fa/getting_started/tutorial.md @@ -0,0 +1 @@ +../../en/getting_started/tutorial.md \ No newline at end of file diff --git a/docs/fa/introduction/ya_metrika_task.md b/docs/fa/introduction/history.md similarity index 99% rename from docs/fa/introduction/ya_metrika_task.md rename to docs/fa/introduction/history.md index 1ea434f248c..abde10aa6f3 100644 --- a/docs/fa/introduction/ya_metrika_task.md +++ b/docs/fa/introduction/history.md @@ -1,6 +1,6 @@
-# Yandex.Metrica use case +# ClickHouse ﻪﭽﺨﯾﺭﺎﺗ ClickHouse در ابتدا برای قدرت به Yandex.Metrica دومین بستر آنالیز وب در دنیا توسعه داده شد، و همچنان جز اصلی آن است. ClickHouse اجازه می دهند که با بیش از 13 تریلیون رکورد در دیتابیس و بیش از 20 میلیارد event در روز، گزارش های مستقیم (On the fly) از داده های non-aggregate تهیه کنیم. این مقاله پیشنیه ی تاریخی در ارتباط با اهداف اصلی ClickHouse قبل از آنکه به یک محصول open source تبدیل شود، می دهد. diff --git a/docs/ja/changelog.md b/docs/ja/changelog.md new file mode 120000 index 00000000000..699cc9e7b7c --- /dev/null +++ b/docs/ja/changelog.md @@ -0,0 +1 @@ +../../CHANGELOG.md \ No newline at end of file diff --git a/docs/ja/data_types/array.md b/docs/ja/data_types/array.md new file mode 120000 index 00000000000..808c98bf91a --- /dev/null +++ b/docs/ja/data_types/array.md @@ -0,0 +1 @@ +../../en/data_types/array.md \ No newline at end of file diff --git a/docs/ja/data_types/boolean.md b/docs/ja/data_types/boolean.md new file mode 120000 index 00000000000..42e84f1e52a --- /dev/null +++ b/docs/ja/data_types/boolean.md @@ -0,0 +1 @@ +../../en/data_types/boolean.md \ No newline at end of file diff --git a/docs/ja/data_types/date.md b/docs/ja/data_types/date.md new file mode 120000 index 00000000000..d1ebc137e8f --- /dev/null +++ b/docs/ja/data_types/date.md @@ -0,0 +1 @@ +../../en/data_types/date.md \ No newline at end of file diff --git a/docs/ja/data_types/datetime.md b/docs/ja/data_types/datetime.md new file mode 120000 index 00000000000..2eb9f44e6eb --- /dev/null +++ b/docs/ja/data_types/datetime.md @@ -0,0 +1 @@ +../../en/data_types/datetime.md \ No newline at end of file diff --git a/docs/ja/data_types/decimal.md b/docs/ja/data_types/decimal.md new file mode 120000 index 00000000000..ccea440adfa --- /dev/null +++ b/docs/ja/data_types/decimal.md @@ -0,0 +1 @@ +../../en/data_types/decimal.md \ No newline at end of file diff --git a/docs/ja/data_types/domains/ipv4.md b/docs/ja/data_types/domains/ipv4.md new file mode 120000 index 00000000000..eb4cc7d57b5 --- /dev/null +++ b/docs/ja/data_types/domains/ipv4.md @@ -0,0 +1 @@ +../../../en/data_types/domains/ipv4.md \ No newline at end of file diff --git a/docs/ja/data_types/domains/ipv6.md b/docs/ja/data_types/domains/ipv6.md new file mode 120000 index 00000000000..cca37a22458 --- /dev/null +++ b/docs/ja/data_types/domains/ipv6.md @@ -0,0 +1 @@ +../../../en/data_types/domains/ipv6.md \ No newline at end of file diff --git a/docs/ja/data_types/domains/overview.md b/docs/ja/data_types/domains/overview.md new file mode 120000 index 00000000000..13465d655ee --- /dev/null +++ b/docs/ja/data_types/domains/overview.md @@ -0,0 +1 @@ +../../../en/data_types/domains/overview.md \ No newline at end of file diff --git a/docs/ja/data_types/enum.md b/docs/ja/data_types/enum.md new file mode 120000 index 00000000000..23ebe64773e --- /dev/null +++ b/docs/ja/data_types/enum.md @@ -0,0 +1 @@ +../../en/data_types/enum.md \ No newline at end of file diff --git a/docs/ja/data_types/fixedstring.md b/docs/ja/data_types/fixedstring.md new file mode 120000 index 00000000000..53092fcb884 --- /dev/null +++ b/docs/ja/data_types/fixedstring.md @@ -0,0 +1 @@ +../../en/data_types/fixedstring.md \ No newline at end of file diff --git a/docs/ja/data_types/float.md b/docs/ja/data_types/float.md new file mode 120000 index 00000000000..d2ae6bd11de --- /dev/null +++ b/docs/ja/data_types/float.md @@ -0,0 +1 @@ +../../en/data_types/float.md \ No newline at end of file diff --git a/docs/ja/data_types/index.md b/docs/ja/data_types/index.md new file mode 120000 index 00000000000..c9f29d637f3 --- /dev/null +++ b/docs/ja/data_types/index.md @@ -0,0 +1 @@ +../../en/data_types/index.md \ No newline at end of file diff --git a/docs/ja/data_types/int_uint.md b/docs/ja/data_types/int_uint.md new file mode 120000 index 00000000000..3a913c9328e --- /dev/null +++ b/docs/ja/data_types/int_uint.md @@ -0,0 +1 @@ +../../en/data_types/int_uint.md \ No newline at end of file diff --git a/docs/ja/data_types/nested_data_structures/aggregatefunction.md b/docs/ja/data_types/nested_data_structures/aggregatefunction.md new file mode 120000 index 00000000000..36544324d2b --- /dev/null +++ b/docs/ja/data_types/nested_data_structures/aggregatefunction.md @@ -0,0 +1 @@ +../../../en/data_types/nested_data_structures/aggregatefunction.md \ No newline at end of file diff --git a/docs/ja/data_types/nested_data_structures/index.md b/docs/ja/data_types/nested_data_structures/index.md new file mode 120000 index 00000000000..a5659a9c5cd --- /dev/null +++ b/docs/ja/data_types/nested_data_structures/index.md @@ -0,0 +1 @@ +../../../en/data_types/nested_data_structures/index.md \ No newline at end of file diff --git a/docs/ja/data_types/nested_data_structures/nested.md b/docs/ja/data_types/nested_data_structures/nested.md new file mode 120000 index 00000000000..653a1ce31c3 --- /dev/null +++ b/docs/ja/data_types/nested_data_structures/nested.md @@ -0,0 +1 @@ +../../../en/data_types/nested_data_structures/nested.md \ No newline at end of file diff --git a/docs/ja/data_types/nullable.md b/docs/ja/data_types/nullable.md new file mode 120000 index 00000000000..0233f91d954 --- /dev/null +++ b/docs/ja/data_types/nullable.md @@ -0,0 +1 @@ +../../en/data_types/nullable.md \ No newline at end of file diff --git a/docs/ja/data_types/special_data_types/expression.md b/docs/ja/data_types/special_data_types/expression.md new file mode 120000 index 00000000000..4cec632b416 --- /dev/null +++ b/docs/ja/data_types/special_data_types/expression.md @@ -0,0 +1 @@ +../../../en/data_types/special_data_types/expression.md \ No newline at end of file diff --git a/docs/ja/data_types/special_data_types/index.md b/docs/ja/data_types/special_data_types/index.md new file mode 120000 index 00000000000..f3ca4a47f98 --- /dev/null +++ b/docs/ja/data_types/special_data_types/index.md @@ -0,0 +1 @@ +../../../en/data_types/special_data_types/index.md \ No newline at end of file diff --git a/docs/ja/data_types/special_data_types/interval.md b/docs/ja/data_types/special_data_types/interval.md new file mode 120000 index 00000000000..6829f5ced00 --- /dev/null +++ b/docs/ja/data_types/special_data_types/interval.md @@ -0,0 +1 @@ +../../../en/data_types/special_data_types/interval.md \ No newline at end of file diff --git a/docs/ja/data_types/special_data_types/nothing.md b/docs/ja/data_types/special_data_types/nothing.md new file mode 120000 index 00000000000..197a752ce9c --- /dev/null +++ b/docs/ja/data_types/special_data_types/nothing.md @@ -0,0 +1 @@ +../../../en/data_types/special_data_types/nothing.md \ No newline at end of file diff --git a/docs/ja/data_types/special_data_types/set.md b/docs/ja/data_types/special_data_types/set.md new file mode 120000 index 00000000000..5beb14114d3 --- /dev/null +++ b/docs/ja/data_types/special_data_types/set.md @@ -0,0 +1 @@ +../../../en/data_types/special_data_types/set.md \ No newline at end of file diff --git a/docs/ja/data_types/string.md b/docs/ja/data_types/string.md new file mode 120000 index 00000000000..7bdd739398f --- /dev/null +++ b/docs/ja/data_types/string.md @@ -0,0 +1 @@ +../../en/data_types/string.md \ No newline at end of file diff --git a/docs/ja/data_types/tuple.md b/docs/ja/data_types/tuple.md new file mode 120000 index 00000000000..d30a8463aeb --- /dev/null +++ b/docs/ja/data_types/tuple.md @@ -0,0 +1 @@ +../../en/data_types/tuple.md \ No newline at end of file diff --git a/docs/ja/data_types/uuid.md b/docs/ja/data_types/uuid.md new file mode 120000 index 00000000000..aba05e889ac --- /dev/null +++ b/docs/ja/data_types/uuid.md @@ -0,0 +1 @@ +../../en/data_types/uuid.md \ No newline at end of file diff --git a/docs/ja/database_engines/index.md b/docs/ja/database_engines/index.md new file mode 120000 index 00000000000..bbdb762a4ad --- /dev/null +++ b/docs/ja/database_engines/index.md @@ -0,0 +1 @@ +../../en/database_engines/index.md \ No newline at end of file diff --git a/docs/ja/database_engines/lazy.md b/docs/ja/database_engines/lazy.md new file mode 120000 index 00000000000..66830dcdb2f --- /dev/null +++ b/docs/ja/database_engines/lazy.md @@ -0,0 +1 @@ +../../en/database_engines/lazy.md \ No newline at end of file diff --git a/docs/ja/database_engines/mysql.md b/docs/ja/database_engines/mysql.md new file mode 120000 index 00000000000..51ac4126e2d --- /dev/null +++ b/docs/ja/database_engines/mysql.md @@ -0,0 +1 @@ +../../en/database_engines/mysql.md \ No newline at end of file diff --git a/docs/ja/development/architecture.md b/docs/ja/development/architecture.md new file mode 120000 index 00000000000..abda4dd48a8 --- /dev/null +++ b/docs/ja/development/architecture.md @@ -0,0 +1 @@ +../../en/development/architecture.md \ No newline at end of file diff --git a/docs/ja/development/build.md b/docs/ja/development/build.md new file mode 120000 index 00000000000..480dbc2e9f5 --- /dev/null +++ b/docs/ja/development/build.md @@ -0,0 +1 @@ +../../en/development/build.md \ No newline at end of file diff --git a/docs/ja/development/build_cross_arm.md b/docs/ja/development/build_cross_arm.md new file mode 120000 index 00000000000..983a9872dc1 --- /dev/null +++ b/docs/ja/development/build_cross_arm.md @@ -0,0 +1 @@ +../../en/development/build_cross_arm.md \ No newline at end of file diff --git a/docs/ja/development/build_cross_osx.md b/docs/ja/development/build_cross_osx.md new file mode 120000 index 00000000000..72e64e8631f --- /dev/null +++ b/docs/ja/development/build_cross_osx.md @@ -0,0 +1 @@ +../../en/development/build_cross_osx.md \ No newline at end of file diff --git a/docs/ja/development/build_osx.md b/docs/ja/development/build_osx.md new file mode 120000 index 00000000000..f9adaf24584 --- /dev/null +++ b/docs/ja/development/build_osx.md @@ -0,0 +1 @@ +../../en/development/build_osx.md \ No newline at end of file diff --git a/docs/ja/development/contrib.md b/docs/ja/development/contrib.md new file mode 120000 index 00000000000..4749f95f9ef --- /dev/null +++ b/docs/ja/development/contrib.md @@ -0,0 +1 @@ +../../en/development/contrib.md \ No newline at end of file diff --git a/docs/ja/development/developer_instruction.md b/docs/ja/development/developer_instruction.md new file mode 120000 index 00000000000..bdfa9047aa2 --- /dev/null +++ b/docs/ja/development/developer_instruction.md @@ -0,0 +1 @@ +../../en/development/developer_instruction.md \ No newline at end of file diff --git a/docs/ja/development/index.md b/docs/ja/development/index.md new file mode 120000 index 00000000000..1e2ad97dcc5 --- /dev/null +++ b/docs/ja/development/index.md @@ -0,0 +1 @@ +../../en/development/index.md \ No newline at end of file diff --git a/docs/ja/development/style.md b/docs/ja/development/style.md new file mode 120000 index 00000000000..c1bbf11f421 --- /dev/null +++ b/docs/ja/development/style.md @@ -0,0 +1 @@ +../../en/development/style.md \ No newline at end of file diff --git a/docs/ja/development/tests.md b/docs/ja/development/tests.md new file mode 120000 index 00000000000..c03d36c3916 --- /dev/null +++ b/docs/ja/development/tests.md @@ -0,0 +1 @@ +../../en/development/tests.md \ No newline at end of file diff --git a/docs/ja/faq/general.md b/docs/ja/faq/general.md new file mode 120000 index 00000000000..bc267395b1b --- /dev/null +++ b/docs/ja/faq/general.md @@ -0,0 +1 @@ +../../en/faq/general.md \ No newline at end of file diff --git a/docs/ja/getting_started/example_datasets/amplab_benchmark.md b/docs/ja/getting_started/example_datasets/amplab_benchmark.md new file mode 120000 index 00000000000..78c93906bb0 --- /dev/null +++ b/docs/ja/getting_started/example_datasets/amplab_benchmark.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/amplab_benchmark.md \ No newline at end of file diff --git a/docs/ja/getting_started/example_datasets/criteo.md b/docs/ja/getting_started/example_datasets/criteo.md new file mode 120000 index 00000000000..507dc68cd62 --- /dev/null +++ b/docs/ja/getting_started/example_datasets/criteo.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/criteo.md \ No newline at end of file diff --git a/docs/ja/getting_started/example_datasets/metrica.md b/docs/ja/getting_started/example_datasets/metrica.md new file mode 120000 index 00000000000..984023973eb --- /dev/null +++ b/docs/ja/getting_started/example_datasets/metrica.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/metrica.md \ No newline at end of file diff --git a/docs/ja/getting_started/example_datasets/nyc_taxi.md b/docs/ja/getting_started/example_datasets/nyc_taxi.md new file mode 120000 index 00000000000..c47fc83a293 --- /dev/null +++ b/docs/ja/getting_started/example_datasets/nyc_taxi.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/nyc_taxi.md \ No newline at end of file diff --git a/docs/ja/getting_started/example_datasets/ontime.md b/docs/ja/getting_started/example_datasets/ontime.md new file mode 120000 index 00000000000..87cfbb8be91 --- /dev/null +++ b/docs/ja/getting_started/example_datasets/ontime.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/ontime.md \ No newline at end of file diff --git a/docs/ja/getting_started/example_datasets/star_schema.md b/docs/ja/getting_started/example_datasets/star_schema.md new file mode 120000 index 00000000000..1c26392dd23 --- /dev/null +++ b/docs/ja/getting_started/example_datasets/star_schema.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/star_schema.md \ No newline at end of file diff --git a/docs/ja/getting_started/example_datasets/wikistat.md b/docs/ja/getting_started/example_datasets/wikistat.md new file mode 120000 index 00000000000..bf6e780fb27 --- /dev/null +++ b/docs/ja/getting_started/example_datasets/wikistat.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/wikistat.md \ No newline at end of file diff --git a/docs/ja/getting_started/index.md b/docs/ja/getting_started/index.md new file mode 120000 index 00000000000..1acedb0f03e --- /dev/null +++ b/docs/ja/getting_started/index.md @@ -0,0 +1 @@ +../../en/getting_started/index.md \ No newline at end of file diff --git a/docs/ja/getting_started/install.md b/docs/ja/getting_started/install.md new file mode 120000 index 00000000000..60aa3fb93a4 --- /dev/null +++ b/docs/ja/getting_started/install.md @@ -0,0 +1 @@ +../../en/getting_started/install.md \ No newline at end of file diff --git a/docs/ja/getting_started/tutorial.md b/docs/ja/getting_started/tutorial.md new file mode 120000 index 00000000000..8bc40816ab2 --- /dev/null +++ b/docs/ja/getting_started/tutorial.md @@ -0,0 +1 @@ +../../en/getting_started/tutorial.md \ No newline at end of file diff --git a/docs/ja/guides/apply_catboost_model.md b/docs/ja/guides/apply_catboost_model.md new file mode 120000 index 00000000000..dd36e885974 --- /dev/null +++ b/docs/ja/guides/apply_catboost_model.md @@ -0,0 +1 @@ +../../en/guides/apply_catboost_model.md \ No newline at end of file diff --git a/docs/ja/guides/index.md b/docs/ja/guides/index.md new file mode 120000 index 00000000000..162dcbc3b8f --- /dev/null +++ b/docs/ja/guides/index.md @@ -0,0 +1 @@ +../../en/guides/index.md \ No newline at end of file diff --git a/docs/ja/images/column_oriented.gif b/docs/ja/images/column_oriented.gif new file mode 100644 index 0000000000000000000000000000000000000000..15f4b12e697ac40c60bf77f964645316410da946 GIT binary patch literal 45485 zcmb@tWmFvB)+Y=h2^JuM;DHc=2Y1&5hY$!Z3Enutp>fy7p>c=C9U6CcZ``eMcY5-_ z^W0}<-ZghVz4f8$oL$>a)m~@S-u2s3GLpP}`c5xiAn$k~A(D^~RY(X?QSE9NV%r^Y zn2A{TMzkOy>O~PYqR-pg2o@y7EbGbg65?hS(c=BMW{p@?vv5$dAPiHQgjq^HTr$NeM(IXPl2^Lc`#Y5f#&ae)|pb9J(Y zP*+Dh9%yT-BlgtoT9*;Q-iXuooAbSl$sGg^4q`tNQIq!^85!LigLs2n-)((+*a&HJ zKpgubO3{~3uC6vC2YO_jp|uEk^r@-It*y-;-{trB_f3opT3VW8q9gnJ`~KKiRaaF6 z1^OY*5)k)Wh^w*lyGO+8Z^UIEq8)v)j}@U!JHE4tm={Itcq9HE!$^NmpAeXe-+rn{ zh$($zXJ$n~MnXdROVUtKqWG%sp#S$ODB@zPZ)jocKw)5PYGy4&dD7HINnvIrM5)fD zz^Y&?VGJ;ncC#~9aZ~(h=w@NaXGAF?Od;sP?_yDib?<*2OE2UjV*z1)YMvp_%nR zMEZY8DJbyESlc`3TN@h7d=;YntB2Xl%!r?pk5@uMT#}DRl2x3I?HiXkD>ti{I42hu z8xK1#r#RogWxv`OI$9Z9JN#SL=zqzI{YTk<0>aAnujQ}Cc4kh-Mv`_mRuunQIKSC{ z_J!v^>iaKQqyOv+?|+nK`RfeJKd<+Hd)@yg{f(i2hW{PBe<%MP{>IjSW8Us>a3dq0 zpB^9X?{06duP!gn&rVN{j}8yu`+K`P+gqC(>uakk%S($3^RT(undzy?iSe<~k>R1i zf&RYUp6;&Bj`p_Jmgc6$255aeX-RQWA*3KbFE=MUD>EZKEj1-M37nV^ z9~T=F9Tf?R2oDPl2@VPj@b~le@%Hlca0j}%x;Q&II@sI&v9+`cp;uhthXN1$jAH8EGlWZ(k+E#Y9Dfg#-oo`FMG_xi~r4*;rYa znHaw?(9_Y;ic-CERxwa!~z4so^cF zc$q#*L4<<)5@6?WDp1>xd*0X6m#!ZxMVJ+qdT7tFJRHHQTyZvU8Uf{$72>W!t=1d2 z`K$u|nYg1cz=eev&OOA~l^CkF5pKtFjW1HJFhId1cSVcfjpkf@414H}Ag*AavBke! z-yq@iALdm&^WTZ(u}a^1jLMGK5(eyWxv-c2Noh_Q4D5dM`DHRU)_6(7hq*IDzeH0% zwb<-2)Ie<1FBt2+QC5E}S-m8ROq?)A)+ttMg@A900I`AsYkkE-IJh+#3V?%b2ERiYP0mX zSs_02ejd#e^QbmpWKtl}yUtYm?-kL3K^LX^r2ZD1rm6fkg2qZ1b6H|aB$z$1*d`_~ zw_@GsXv`E49^c+?7oC)}fxcH#hLnnt%YAMT z)3-}t-8T}_yZ({Ly-!8>!t`ylWWk@){+wKYhrzmCJlmHyID%jKr#~9Q+a;#HF`97) zdr%uo7$vB77H$z8$}!pmRM+-2)VH5-ZxQW!Sv ziLpZ4Xp?)=^{yE3=8<;R>4baOwq$HpWmaJAd1-HND~i>AZel5@=(P-BI^eL{R!V|ed ziXpG*o3{$zRr01*PUJT1QinqZFS#RziA~>EMXAz^;<3Lm+xmIozVE+v&d2e`s#j^= zAz}h{^}FWS{YLMFR&Wqa09;t>!Sf0ct)K#7?{d6Lqu4e%mO&3cX z8y<+;EddT;&RvlV&%48q)t($Dq#wN=E>Sqz*l%vqy`CO-tJ)sc4l*d7Y1?1)`g-?> zQo!E>+Fuv>`gD3{Zd2{GV}N{p+uA8wH~}5lcD{biNLi;Z_BwFyBmJ8sJ|VsWIte6w z187Sq-snnqeH%U{;L-VxZP70wei%g{@?QQeaIc+^=qy-z3mGS>|0@;f_e+KF?Dt8x z-Sk=$!K%$0_|>ql)QZtzTH%yLU4S08n96V^!JPLaM3UU(6JZweIV6R2QtXYD5ss;w zWR+kkq3a(Jz}%dVs4~4oRAV^7^0}0R1Jd8leQ+YebE)3l^~-2g#UwOSQE?1N%iC2U zqSGI@s3hnIe^^(=r3o6-%QVQUX2-;o4O4yj#y9kRmOZ+D%ZS;SSWf1cJs$FyM~QPM z`J@VaUQ)Ic2einwM~2LBu5%JmGK5 zPsTSYoq(5AD!(gG*`R1wXP;D_TEM^UI_9Bvk#T%TD|q2LW=B$!F#yaJCmb9PWaoU% zW()aBdoLZVe3?zqLMQpTXd=+0CI|b8PMV%!GCC$Mhvte-RzXWSp^P&R-=k1cn_oHE zcRC-V$4ttRUnNN~u7E*-UI||d)#1E4{h4Zyf46WfmsvHlxrA0FD^#wiiX&wSQB%ZM zJ3n1Yu9~V5Q9u`wteWdKldR>)pfO}VTjr@!^wX+H8)s0>yS28=sfU)U=U%2NyS7v- z%|d4iK3kluS}Lnxsn^vw*Hads?$u+#OZuRpNb^2ltjE%XO<+3KZnl8y)sZRJgUUpp zQ%&-brCEUaLPShLK@pajg)VSvUU<3=Q`PE6M9_36WnC30&C(VhQv-^v4(%hO`xOG5 z>*wbw8k(UNcePkrdCgsFYHVpoZ?OzFziAo{ICeq_)j5?^FK@>$H!ggb-Vk!I;SK87h%FTdQjV+EcftUe9@Mk$!> zz!5$Q__erz@fq4)WLy!9*SxqkGS_s6RpF-cX7%A&y?xTEJk;=T^>)FzcQAm>`(aS; zZV=it(sLH-;l4qNlhh`?!WI^}^owE<+$)1^NAml2*JrH3BM^^jf1$(qSM^Ca4!sv?B8SS@4-KPsAp7r{a)77=BA0J8 zRZ%^R+g}9S2J{0NbBR56zR5LCB(z?FM;^DXl8JgknK=puKJ3efxlN~eau#`|?9e%1 z&Ah^=)9qU@{QO;Os))w1xXJQBrC?!NEuS<0GuD9;Dsbj1t?FzC(Y!m-$vH;A?2viH zs4v`gH$M_sm!21KB+}%%u#{gK*jG9sal<^(B3W1V=g2}oaba0ifv0)fP1jLUR^)EyIXZCvxP&vX=+{lFcowRiy77vxy@zM?Pl##r zCIy1GU!v?RT3cs}rj2({W#ufsx_O)0>u$jC*)DM|b%*_icT~#SKH)}iTO`9}TtV0& z{UmksTSmihtni=LUj=t%gWRUGMjgN^PkX|B_cM7Qhf-6I1LbnJxx{0~82gqz19jK= z5$wxab-^Rx+omZCD_h^y#bb=VruJ+{N) zo7ZF4yHT3~7Wb21%-pM_Uao^*1YC6bpo<~GZnZ(e$9aSvd+XL$Q*A<5q^3gfG3|ZAav*%uF7gc5FMot5>fTicwNB(9 zuHrC1PGqh;R$z^}2GV}Gh-DGE-_<>7v}iqct`}TGmA|>@%Q#E_j(8$0N8F+6RXpx` zC8@h??Ak{0$oF=+!{&g$3p|!-Juvp%4TId9O+DWCd+?e1j3xTuf<1DBd_L}Ze0TPF zx99x<=1nK-!1p%gpP;isnlS_2A$|*sVHSucLaMn|g`{d!@~JJ=A-O zxcI|)JW+UUQNjL-E-n$dF445!J~#eq8va}A0qW8=B9i_(MEMy^%4YK16a?l9+Hts{d2eg0$RnGaqXq{{cgY)YBps`5kZ(jxm2M2+J z!=b^T;9w2!ppm{nMpR_tw=coqkYs2`G8p;ocpx>@mk}+LD-rpHZ72jBS_BOx_73uu z4&jyd1NH}5-gwu*{`k8H5LJXV!@^o`!&-=tVewv?5NWEn$nD_p4(S*2MBd`ifPP5$ zWH53kEI7~H~`dO%pOx*s1nnzS(b zLth?@@9eE`S(gha#>#vcbuuk6L`@rJ9z z;UEEP{#PQ1nVe~lB>wvy*@nNR%CL!KWw=qb>F>jJ$KC8z#&?C{A#d?M$ z@}WC0Hzc|f$MU#>In98KcZs3I2rzF$ESU_rgf1ip3RW&ibeK>490JB@NMhgt1I-e{ zHDeR#V`I_4RkvUsNHSqxY++8k_P*Y=RNOcEgiEx*QH_Kh{{(&X_^Me?tB}aW+my$| z_=!WNJG>QVelaTvAYek{$RiCRsB% zts!PJ#33^&DKH6q$`|`)FR8F0`C2n0%1m$-Jr*S-`Ga~ojcfYBK+ zVJxOr(wUa8-ctV*G;)JoJEGU)v&zM|51<#tG zm+`!h@Wx0RB8KpU$_0iN2Frp%wF>{Z`h#SP^zPG?L-W;>?UC-IjkO9)wDKI__J&2d z82km_-7>IA3bY1`+Zu}%5{qIQili3usa%U4??OsKOHBETD+@~A4dmAj=Cp#_ME-~**~0Ka&{ScLm|5ihb0AaYd?Un`i$`+S0*q0aO4!f094VHGg*-dICHo)^6-Ad<^(`U3Q ze-1|9%j8m^=i?K*;mGB`8>$$E7k>n1TEHur9;zBcOW7^*I7q7~oC~sI3kbE!l?SA6 zio&-Cb12-(8Z{x8vXMD(mtzLYQ@H;HM#@?;1jbb}piy%VkAIm`gSd}HCarmZ=X7eu zqo%~6lP1$Fmb_1_LTjiRgIAGYmSzxF14!}@%&MOoN*%uZ5d`v4X;(=OR#^cnOVH}= zQmXMV5hWAm^|-~=?@3Fr7V9~~s^h;j=)y}0v}&{6YJXvdYsrD67HeP0$z-I|>W6tL z4Ap*@D-sGVN^Yt-FfT9o5_!kanD>yiFI!tURCcUhXX9FD1Q2|u)Z7bf?oVm%D@GpS z)?Z6d7Rp2(6lfV*M49WgoFUMkzP6l{Ygr*h9&T#cBCz&&4e*_{s{Piwp=Pa89H087 z;=sQ)6WGKHZ`>nMuO&?}Ols1bS7>iao8!~%*sMPJQl}#jcNdm=Y0;h++Egy5i2fnG zg)|4N*}7D&?B=Fpfez`#wwvHi^`ws7?v7uF9pWK9b#lF*hkK-pYiNgS)EJEzb$XfQ zJ4inWVvqNEKK6Nk==WkoRxL>l{fR=3gcRD`AHLKdeu&Jq)~lV`*)W6}a@Y?JA0S8S z%`NUwxoe{*Y0s-M5pwT_Yqu7Jcf(Rz7HL``mR&N8KU}$olMWXj3_Pazn=rPoIS;qJ8R;k)p3>>iO&LBk?Z0yA!&(~7(Ls1U zy44^NjZQBO>@){BG7hFTdM#O2z=sVxst1L21}g+x>)pq$+*^xM$8Has?=8okAIFfG zhQ8~J|6mNjwj7QbZe-CKosl1bgCAkS>l*HlrrU;5ntD{y*Bw5(P!hQ>=z^(R zxFTbOsyJ`U?Kjka5ksvV8*zM1%J z<)Sk(ivLu`tJ^{RG|{6or};Gd@y(PU^%QaV%(K8O&YRiv*MjGVgHlgZ4b6z@2OU^w z#5BqiOmZ0(Tsr-t1ty;cOAz{@l*X^zGD&SUWp*@4(K7mKarO*jVfe$GIL4@J%S3L( z0t?fE`N&*R#IWIqNdt|=ij+z3qdDW{Ne!~a4xxqXxjCnn5tj&a-zQgpp|S9^>7Jvqv^{>s|l=?Cc(MSmJ80yi(RF2=sct8WU1b1izk9GLFt*I zheg%paZ`m(Hid;UiJETqe}u~y31Ut z%d-*dQKjn&WGhP^(>0IFgr3VVk9jiB4cyv!$khDl!~8(n_+by)&YqfTx&^uIWQqofUEVRXuzJk#@_b4>{{4~ciI*ZlXH(~{& zJYLqd-XO(Z{(!whcf37Awjr?GEuZ>ZiE#%rZzHQ~8kSR+QefM5@@@p zVT*NTt9oR!FKw3nx#QdMxSH^`Puez#{7Nk2UeL;dy6&DLb6iOJ@LSNPCumaJdWb-8 zW3F^(zSwH{Xno^o$2fhRYGtsn3|>A8|ERcw^=dcq^X`?xYKPVCCH0nH*{;gSuDY=6 z56pdXtHnmWy=KK<0bKjglzlGHtaIvCtkEHk)mB5uniBb@t0H0xbiAm4yFvN1dv>_A z)4FUzzEiUTuL2z(f#B8T2m8!BxzF36mg9owomujQQ0rCZmcyW#!)4Ik>dI#Q$lkk^ zeSL*B|CN)0(Q#z2-PYqnUW}t7hGQ4THQCn9aO`7dy|sAI?!nOsOmVV{IliaWau*wZ z+6q4xKDjL94hdr=K$pRXtC46HmK^ z*P(s~twR$;h+fK%qYmi@ouJdy=SzO(SuB=)l<)hkEu$i*AugM`mleb)Q$Ki4ln6&u3DD-5)likA_IdW z*JFC8nK);!`mZNH&P@Nh|G0WO;B^HA-Kmt_jjdiyAg-w%GGu1>0I*Xj5}_ zV@u25lC66ew6nLqf1r5?F+9>U);BRZIy^fD8=q{LURr2fS>2e~TAbhM9;-Y&IzBl) zUYfeNyjttMZMuhF4emeRBI5zV)fZVGa44HS&0>Fb!I(xSuLp*f8QsJ;$w*37S-QeW zMfns-CjP6pKfkM%zn&=y{xVaj)8hH+U^-f(*tky*_o}*7yPm-U6z4id)!oQo)msZx z-i3m2EHLs15;sPSd@vu_ZtE&H!+{sruS+z_#RuY1P0}9-SvtENSo{PJ`7#W~hslfe z$i7rrOy)f~c}64|BX$NdzT4S-$qw>kJc*naWbp~%KKIm6oT^0MOqJ=&O>IStY2VH zt@sqT)UUcfzC+FMjhmUH@Oy;~&Ghq7kk*&MH#d3bOVLTa9!z4rw;q5j2{8yF%I?qd zqS+zL31FIqttx)n*~7C}KBEh91tZa?I}Pgd5>NM!bPH3XaH->rt*)eaDvb1ZajrxLn`u?kR-Qbg!pduc@%&ylUDqGHwP4xw3n5Dy-<*x+CfK`8q6@b2PwY=Y&k1v4NMmb zB{NXFstOmvOSD@S;6*Ey_l6})S&4a32hj|c3h4lDa>((tIX4ZrQLaEFu2#@>-h( zeaN@ zR{l&KTCQ@DN;ow+O@%qNM25y|&h+IC|ca zbPzGLVZw7(J)sQ;v=m6%_M= z4RLDq`ez2(MG*KE?psgP)XWA-B5`~*R&U>?wd_oFu6^TDw;!L#vAlws4?T_or;58T zoLp+Nmf`&0XHoMkFQOyO4AiX;|}NbQY4seaz;WRuqJa4(++m zd*>a(2qyo*EB;XWHmV*k7Y%)Yn)1a+poqJHssprZGXOV8&>%h9fVEqhq|(&@a|1WT zNtUV?H?;1MoVMPm|Lp3uNZ2qD85uEOpQ;_3oggXo*=_$)8k;s$BInpCDMzoh5`?Hj zqv;F`krr?|`#LCCpS-bAvq|Rd-uF3ssgTbTS(~FGixgU_#rCGZf`pl)#tx zPR)LMP(%r>NG0(A?oMnvlR}ow?|X5_)^VIq`1JT5=rUTVHNTb^&E(DC3P!Ed#vVp^ zG}M5p_*u#VRFXpBNX@Ve66QSdTBGm!bOWq0_8B`4JIdOy=?X=>qJDb7CAM345VlQr ziv<90w{b#K54Ehd7$9jMG8ma%RPMPA!Mnpyt|6By`r2Y3*h9}=qC8W=ID;@}C9;5) z6X5@P`f#v0SOhB|xvmgspyo%f<9D2d!Il563gNx{BKO%9+DJ-O2yzdng#e`R9! zv!M zx05ZEEyYyT-&dwX=Ka41s9WNt46uXNJ}k-XB$$*tg9h9 z%dz4p^>dxr_U_TT2b!NB#^bmhl35<}0jqH^-9Sg9EHVp|6aI$P9mgskOf#z|l9G;5 zjyj;P*@a!>lm>Hc^EaWBb34L;v)&uXk0WdUoyEZeU7qjZSP1Js2lq>c*LBb#A^pG$ zymhWxC%;!K)K+-0i(Gow(4*#4@(ZGodTh?pDJDQ@8~^C9YnJXvvhxT5fu$-ZsHVEG zMGQyp7Li$f!BNXa-6U`(bM(Bau-RUXSV&!R>!u(;5iVInvRhgSO=T0-iU@1oR!?*- ztu50}KA7L>@o^JGBeyFWnqOmKfesO3UDP(E9raz^_fsERRz~W==}|M6h{<+JfFBCI zf_`Ts_S-8Km&y3sZpKqaFMH>J`=Y3~V~7!#!4&>|u$Q3Y)zM{Fo8KXRo#jv|=-Q-< z?8wCQZpK5%g<{HMBHiZwV4l#!R2$hrorufYdjAy^Q7@eN8=-WlkL=c(*t+Ry-L&7e z{GiK|aTQbMv>qVhu1+Pq7r)!QxgO!s|6TYYjJENHY0M+TB7Ju&Bj#FD_X#rBx>op| zXYN+_?3c>Ob_EJn={)Xfa)+fDAXl zoOq{#8~6GGpEoe8#{zG00-sV_5BWMz8amfi9?#_<-+ffi#a)|SR6jVO>q)*jeLw1V zTHoBa-b5N+Xb?bsfs@Fd4=EJr`QF2nK!m>^Ll6u+D{vEb@opgWhtRscKr;j0_)P73 z$$>ExU{2p5P8U-AKl|M!p#fiEKE$*h!43#}UGRsuwSuzKHXG``PEcQFjUb+W-|ZV; zw_s1_TVHMs8$PK2FR<51pQRz-EF!=Mh}YL!Qyd}*8yAcqjX)F^QUVtzA%IsT z3=qxhmZstFW*-8nK>b^>ErErW^@qMvLMrDCtAT`8YJ@>u!l2-=Dri_4u9rTCoLn6+IB2I!Mu7V>E!IAyEk(a!nN3_UiSi}t-2!%Lm5-kdY zIDE$ilou6pMHrY1_;~5;4&p_GWV-}DeY27YX2-sZ!b1zdEwnV!h#r#k&IZ~2i;$QP6)TK=p&9o8h{N2E zdvh0^yA~@W6L;+rr-mMXp%Jf%9;Y!MzjPZf+Ym3M8E=vlsgsnzsTpM{6N^P3|5h^~ z1QJ3Zlb9hLK!O%Rg03yyFGa!o`y+9pt7|lonKwpI3{hdsJNgid+n84B_#DxA1N4M@ zx>&Ko1Yy2}nEiMKnFOBs1o^wTW8Nfjvm^`61c*$y4RLI0LsGg~a@t*TSyEEoUD7Yt zggW}<#=G!yqU5HAI4w;?QY$nWn;r~wmDhAh9FzeQLcn%1F<(_f?(x8%(7n_E?vC@p z%E4f#om35t(04(Btou>bNhwvX=^d`|H3RW*SaKVE>YiroSyFl&dIsDq<6I`AMmlv@ zBfWwzquDhBi6kXyAief3UEeJIJ|qKkAr>V$^UXpeGJk5+JZf)}cPunG+$AP8I7`Jj zHLu=o68#OnTNeF%mMA1GC?q@NE>W=`%(M@#d6o3MpMph_)pD1KR+RI$F^6w3N187Q zN}nZ)k$M-BlT(FyAX%iBnGn!DBw!E{#a|f1Ul_LlNl1poCl`h`7N%kprr#H4h8AWQ73MM&WrY?M zXca{_79}JXWndJiG86~O7AKh(m&1!9LyPmZiki(KK&337{|_(+8R;)n2NlKJ%jf?D z?EL>jmqcPf4*shj;qCQ*1^#%wdjBur&#QQ0vh3WP{5(j(+r*MmKdHB5{{Vk#-b$7I z1^&cU*0r~5cqPX$^z{q;1N<4{!{b7oLZ!r+n`>F{!^P`o7+n__C!a>;*yF%iJUmKR zSsUZ~n?+vXaUvsAK1QzgjUJ9@NBL27U~!7lz04|K7xsKKfi_Ju5pw`B|9(`R_W4Mm zVi${bCs<_<#GY6&)x@NZfX-y;g=iHmgX&ZgEj#pT2q?EU3ODN?EZ4)jLX^JT2DS zlJuZ}@j4nFvf~EfBjn<59N z4uW(SCYiH7&o^-AMaOxo-DUqEZyu06305VNs|2|ImD|Ce@pRzF0ZuI4%=v9RC2^VS zIU|ok8rVCjw>vW!cw$^(KN;@dCmFFdzS<1uBHxbJc~9fIfB>1Oa*>$Hfnt)w{qXBv zges^7BAG|3Xo?szFStIBTh`3?YI`25)@yqq0;noXqao z#0RS?W4ZZIZO5T)UhP{?GPEisD0h2dB8Hg~9SBitVyG)qMzO7rs=YOVIz*RB>&m+LfI)8Pv9k%p^MXV8X-v8f8(KYO`SO1FG4wc$aMNONm8p2ud>KZ)EyAjxO=)>I`a>lVSoMxQ%uT~M zGLbvjA{NG7f*9X*ki22iYPRWnP_N}{8M6F=IIaFuT6oqc-fvMo^nKbraEidHrdGyu z(SFO^Q`>df7pJU2c`sCUb$YriMB$Ad8msR5f^$qapUEFzz|sM^DfMH3iJklWuMXxL z0=_2}n~&Drt9<9l;Cu{2^VcO3Tu44+&MioK zQ~un|WmzW{>9Xl#8hfA{tRq(`L!3p?Y$np#K5D+DtWlkvE*X6*cZv6UlEd{?Ouh;s zGrtY+k|dh4dd4KiB^KU`8H*KlNBow6fBbRm#N3h>Cgu5M4ZW~N3V&86;>b3W>=U#c zo#d5W9a1hCTd)-CJNAH&Mqnfv06yL^4kCd3C+yR^1}s9ow}DuLt-U2wBfr+K;L z>B%_>nUSyVmTxxggXbu;X#RNQE(PL!|5nN?`@`+$VG@41sld_tXkyo0Iw5*o?eF-= z#2G$4SRCQ7-Wxot>EvIRTJiGr-|W0Sr&gD^`xPmTCnHlFGypH}WW;-iq5=;tDtvLpt zy1J{EwCx}r3sLn(#jTe@dqwK?{AyLt`la5g#oE+Ja}BdINJ@_1^se}QS_4%vq9P7I zx{3M@swI?Kaam%0mF+IzvoBm;Oq=N}P)^o;2ZUXc;G{6rb?a@U*!)cSP}w+TqhXU< zk5a1i?Lk5(`MPGD@1uM)s{CB5S`oKg3a_c$Xx-i**b^kcI^VM#HuRjXt~X5}ybbEmS!ebTIz4V{Ng(Y0*mWd2P9Gl+ABc zZH>b%!pe=(Zi6fMO22GERRT3K!X(7-cjI4p($vhlZBVH0B#C2Qfn%XEATqcl?*+E}8bHV74_`qC z28z5s#_N82PYc^$hvPS6S4S7=eI}-Yq!eu)uQ`2=!dAOtG)Y~BId!)&)W0D#s|`Md z|!+|N{iYD-KN4zw2UXQ89DWzHi9dJp$= z?bvk{K?;Wkqz|xu#k#6Q#K@tEz{C7FsII0!;mBP3VF5N;S64rBWDR^+T*I!1_9-0O zr93R{Db_d6jT}2QJuIJs>YH~JPTUqBR&Gb@TW>~AJRcrbk#L~xsEVh)q>pQ8-=Up^ zqo;uakLzzEq5l*3^SD7Y2JPh;Jp%zBHz{x$`lS@lV^SWs=)Sww@_3{sNKlP2i(nVw z6qvp9#&LHrCiqJA!pgE;a7R>Rlv-RDP7GW6@a=d^s%&`CjNjt*H>pMm(M*S{2+jHZ zOi76aEys1n?yu`iUvodVm>#~r9eW}C#RIcM^U2BdFsAGa7KYqv4NKz@8o@P;mPx%n zyB1aGwB4AR5r!C6gCYF^Ro6~W?ew{?FahRXUM0~O6;D)pT7{$<{yIPXNlCl&rb0z#~wBdn@vuV>d-Q0T~J#TC%8L>cDo_eo$L4Uhaea1QX=+g&5qeI|hMN3Q@!0j9YB&qh=?8frJQ zX^Xi6gA!WPz8f8Ve|@Seqn1SPoLcX)ac`cVUq4HFl-62wph^)!RLUt|ymaAaQ&avZ z?OI3p6}pS?W)zd0B6OpX^y&Zh^=FT-w2HfIm&Z!I`q8$A@s+2ys;6(lul>1Shqe9& z+n$P0BkLX~)E$fKKF`~FuO~u%JQEERh!10r&ugeY_ntd1n#bH{5B(iK!F*paIzb6K zZSgt}79u~`luwMh-w3U5si<#zEwGE)N8i)}pU(ZTGDr#*1kms$X7?kPR-B_Y_V33C zgmMK-8>p#!h@z3IyfWGe@R0`l|9+)*Jmas4=B*9zj0p66xC(H873g^ts&%UtJ{!2IYcv2*43#>R6t1y577%U+JIL{g`PmP0V&bd)^2~h{YE=?T~_q ztnAQYqHuMzUyabPW}<)^81`eHo@&3p*H!2lPuRt70Pem<7jIy%hRb^z?*Tx_8bEhr z&tVMJ;LmLcUBAK%6eJD^Ws;6M!Q))yRQq5W^dzAHT>&((8hu<6-`6=9C^9&d z1rYTd5M@N2K;RmEu8yKY7i|_2pa@FF9oPA=(j1iL)IJET3^d|4AMTp_es|!H%|X5yxGU0e{>7 zq|EBBQuB5?3i}KYvhHI1Mj!iv)_jJ~*V9$#@s>`V09V}EvZL>l@)|NT5ku-VY7KgV zJY7QKPSo%|kBj*>F)aua*Xe?@P*=0%5Co+2{jO-vBNiTko z!i=#9;g-gq-wJ5uJcVF~vkGj!B_GSooj?-KamswJOu_jXdyo~G{4^70^64iDw!(q{ z^#oe^yL^LtEWB~E%XtM((Jbw{d`mbo=bBYsBIz7^ft^;tAITT-Z_)nu1Wpnu?Ejf~ z`2zpdagq49<03vSH!tN?c46GBSNUZnMSnXkg8z12cqP>|HAi^-MzcLrLebu zq;0r=aB?cBZ*+8gV0y7~Ze?tC)pIFoeq(KSVe24qeSdHC@I12qDyif;|03Y_uAuIj z_lOjV_PZVUTKAn!@IFPMC$>Nqt~8c`Xc#sv+(4Rj{aa)`?6_``s~aqt;CJ;<0p zar0}8++9pQZ_<(?wx3MQZh1U+%iVjD_q6qytAX4cvP{`uqu#M8FHz6vsd}j=sX6@p z$stra34O*BQ2WSIUY@x#grSzS;08e8*KOyifF~2Ay^b53I9MhC_Ug0S0*t3+jdbs5 zLG^PsGJUX6lKfY9m7@dI{gl(q^Ncg~`5P++VSEI8HHvb58;ov8i+ZIQq12^X)BEG~ zGBb*KM0zJ#qTg|8UfbVPb;M(mqRuRio%YZ9-pDKJoS%Q6J~B~v_K4eo+5XBH89{%P zfGmqdCz7$XP|qIx0OlL(sXU$UEvtwAn$7Z*9TA#)V;g~ok&l!dtO|os7Xg=Yz%uz?sOL3%95-$lgf4pdt2@oS2avz!tLDi#u=XuQULv7+mS7oT^p+ zFIHGR>HVJ^XcY>r;&~YsOHoc=yk%*|rXyvF?0SyQrf#}!nhra8scle+x)e+WT@Ov6 z!kVB7JFIzU2R->TA6ik004;lx9`%y4JlaOmDMOZxYm{7#nkbqD%2Cd%4%7 z&;uV=oa57J3Bkl6uBQ2B&wt?K84R+Qw5Mc7W1InVj`b55~9s^L6Tw1E-V2?jxLP)u)|$`y^zg;Nnem=+e%RPK?o@VnP|;?6 zay>X}!2&NX7Jfujm1^5G?n#sEQv|`ahL7PLH^=8EET}jvXg6Pg)UD{s(*5(*ceox5 zsu^`=vH&1Z>qQU^j}R8tpSQf?b15N~Tl7@6>0V(~X5gOSR7tkl97 zvR?9SL&T75m0qO2%EGwis4xAkg84>!)sBcOhi_aV9Ajgha<7(4wL*n>NgMZxic@SU zN--Se)4C@_0v9yyUn8rJfnN{Rp~m5BH6X9R{>o?g$--ssVreJ8fd5eFi&OsQ%Fxy; zZ+^FSl%`nqn|3bY$jW!XJ(T5vmp@p=0R2cstVz8a^;HQII4+Gq;%E4)fk&BQNes7k zLdPmWP0mO}_!vjdWeH&^yp}D(`cmUVJ&#nv{JY4r$WkNZk$yW&D|A#iW_@>&LB2Oj2DE+Q_4+d7q}x>J zYCwsaDK3jkDTZR29?f1+41ZgmR`9?~$%pbX=Xr}xT0%?7tBNCS`>~6Lx)0pnK}Xi9 z^|phtfK9peEFsC>kU7t^Ty6FplJjuRO*5hJSo}%cu3yf+)BKPG&AC%pxDlKBwKOV) zeM~GuXXo^mHY>ih#>v2FpU*&+aWCy%0Z*+N!zx5b?RYd^G$8LwknE5xnQ8>TY7xbs zul-`YTwo=#{GZ#V^7pz|p7BOIAl7-2^4G&Gldgwb;+p-{MUMIa-GiNlg{k^yPDo|$ zLDjvB`cE2sp#0SVO9otcDbKPZqQsne~y04pMgI8bjehr05}%3W1ox5yC*7_tQ)m6)i@nb>vBf?gGAopEkO^RALpPV zdT>qCX%Dl^_JQ89xU_)6U;tT@Tp$qG-^4@USE#@{t%M<+C@ zQ*^2Qba#Gd9sjARl{X9!abBN%m02FB!}MF5bf!KOT0}Kk6mk+^pil%>PYds%MP^i~ z2A%g>G#_e(H8+w?oy7~MQ%4;!1Z$^XI~FRi2WbxF(DXTZDkYd(e#Unrr)HQI*<*HohmywkB}8Q35kw_JTIH#>l{9jXsukRaW)Vt ze3Klg{c&ub*m@Cv0z3&*e!boO?TJfXpZsgZ&CL5@FuuL-m75wz7k-WXQ>Fg^!c^`; z6AZ)Y!d16w+2Y5X9XKDVKB-8C;UVs{5J__3ERw%}lM=;CcyjD*?P+obsmo9?!8v=T};R9T1`u6Wq(Y#7+V9Nt)7gbMtmXs0#9@ozt ztrBPJOJ}5G&nr$=weVee+-9aQYfj)Nq5W&wxyeNAE1}A!eMR8q6h=JXFdNg@IRkwf zmvnj2+z|Y_^=77`jK#y$WAf4ZaZ8feInpHcwB`kHm~akU34F8;>C^7lD8HdTS=I*~ zEN(h8AD(0{9mU+>z_p@bA4#?->p_I}pHi)O#|z#q}MWx`K6^lzE&n4cO2kuN~%!@N_^yZ{j9MJBx~v&V#dTC@fJg6 zskZ#|eGylxH7T~B`IxX0V1T^5d%txz23%3hn7g=X;dxo)(RRL%fynJC=gLuG!r9_R zG{`2_dZdtF7Mwgk#6aC^O_Z9=dOW3aeu3{PU!tF(&|jd!R+TZ!jZ$CIx3?ZM4p1YU zP9MAopL19?gIoobJ$3RZYD7QH;C|YvcZ)@~elJd)a>KTI?M2|?Lv)LVTkV4v>qCC) zLqX(A$?Hp{;Y;Jp8olxQu7_;wH?pKX~>A$SP%a{}SQOq2m7FAN+OBW9`;`w7|Z*&tsX# zr%#%t7`!!=dNn4KU+8iZyJ1GWt?Ui$1sj%p7Kc;qzs1tuW-v zK&Dk{!)UemHFv{}upmV@U(^s=)l@fxR2S+-7vv{5-PE|dl7t}f$Zs#O1CI9Q&<4nl z#xYNJR-7iONr~cVj>izrtXLNDloo7#F}u_;e~T=@hAnCib-2}17&&8RkDPC*;qF&- zSF!XY-BQVtV`W=oW!e&ZQg{wS+yu(qgf&cA$BcejB)FJ-DwmjP=5WT*YedSeI&^WV zDW%(}aC^N`@oHKG`FlhFV3UqgxU)->&NvbxaMUbZ6C^x5<29n@B+^r_ldjN`gBI-` zjFK}w;{6{}p|H~UEi!&+qzR2_gnO7)t-91$sKt6@ST`p%t|m2Gq|=fmw_B)pwP$v7 z*~Yq}X_*5UV&@g*z?&iP`;`(g*DS|6h0{o-_U9F#BV zGNO?|=#j;fk~R9MAJuLe1M3RL?I*)+7HXCqAC&Fpnw%A$oD&}3bZtnBg1s!L8bq9B zE}BIIn`EGtmw0bKf#WdMo=4Lj7s?$u51XH_pIPJbnZ2r!2sF;`fwg^~qkE%36Xa2a z5?3n3$qSM!fXK)YZ7&d8EkJwBT49AjNH+*xNXXMKbQQ~29?K>R$?qx45AV|o<}hDK zG9MxSwi8sc0+AE*2(8DRVQd)qu(o*A<2_RHC_ZY?n1worZApI(w-!7NY)1QT=Z~F z9V^Z_DaH&>w8JWQkGD#C)k~4m5aY=Lr_F=I$>cN1t2uDsh^){K?b#Y6~>={>pfnMw*Kv-joE-gJ9Q|iYKASx}0c!~$c6j#pVwRRYzZjM|VY@%f0padPGlwsM?88>F>XUqY% zcQRB8!b9D`qIqPCR+D`0rE@BDaPHHJ)6qd{m|&$OsKy#?dEP>G@`FhRZVh6GFCulu z&b;`sl!Iisz!VoaIY})lu4a09=6!?AJ100`rj{zQS_U4QfkIgcKG9N`@U1ADLTY-dsbq706!B{x*u9>M`nm|1oyx@BbFX1t!2+(F_VhVM^C61694*Ui*g^-q8y&u>)b#tF_llOihbSsDWhOzUQd4 z_MVPBVy7o8vV}-2Dl1Kr;8-uYQx<}^J#_-U#;P5o${bki7~~&Q*!e9qEbKtBETOL7 zn5Uh~eX~gXwt$qpKJs}_o~S`wkM1-3-ecjx z4XaiT?-oLsp_AJH3GcR+ie4Fn=JA^V`S(UmzJB!c;Th-t&ba<0t@dzC05zYSv38we zR(DoYCqu}v8a?T+s&4J`03~ajb+1aN$p$&CQ9Jx0oV3yJ4Wp_XL(OaXn%1LNAOQU+ za|=FmLm7B0{E_&s61=DpQ}40b$Whyk(G7(`$BiM9w&CcF5#}fpd;0P2_0i%<;0XvI zXah@y9xfRd7=ZxH)1KIQ0w!n4Ci21eB8;JVj}QaLKq?2ysyZtsC-UqkYR-Xm_}vqH zQ*Bj4-cdlBw^1DP@p^5MspYKk8!5yY?@{XGsVyczPt;^KAM)V)1S4=77eS83y;nwI zdgC2%EHg@-R&4`1#eY1-h)ukVJ05Z)DRkV!j{R?Qm1q!k5M5B?e=W3q0RV6Bf52$J z|1?>|h9?JwgeLxn$s#f(FgQ8yUnYx+u%d$6^vd$O(3;|gw)U*Lme!81xk{PW(2yVG z@INMt5kIeigsBNg;QZL^YU#4q;>Psa?q2)&==Raj*2LQJ>G5#n&i?Mk<<-pH%WFb) zTGMCu$BO|4lfIB&%H!@{u>4Nh4*R0ioOJex_?TkSq~0kT%%49l@#&Don7uJr4PBBi9&Jk2M( zgNm`FAC%T+1_LL=`)`#@bvuTU$+{&{Y)-7d=W+GGWn~>r$KMV{nyum==#~v$_H3r0 zS9I$e#s7!}@#LPi)`4CMPLq6)8k=o-pud!-A2g$R z{_v&?dA1NxX}N))jy!ly2MTlDPfs?pOh{=+ zGtf?34%21!Zu^RorillNNhkBlL7^tE`x27mJZVZZQgCVQUX(RY%b?^fQwm}$Xy`H{ z#b~9=5;Y^rOGxcAEXp(X;HYYi1JR&k!QR;wnpbnIcabq44bm8|MY;46}nfp3QQ-a8Jqd zA@5Mk8Djsr%Zt4cTWwe<;hqJ`A35K!6`EG`yyx+w~)?~CrhS&u&8$js-v&uTw z3erpBC6{P3Tia5s%4`1!^ws`O1FSQ&9i2+i_Fj!lSd|Dm!LfBLONhe^PGFy&%thVf>1I*bGy>ae{oUOI>(CkM)klA=$a&F{6 z+g4P@Ve7IlH7VJ2HfC{r(4G5HRN40(Aijt7gMt4xVk0u6X9kiRo_CF6HJ;m#6P9*4 zB1Ws%@|Vtdo?ipEe<1XHf=0$6_>vuDMm3C|J6#D(P*Z*K7f@>E#8Ke~`@l};pPsuP z$yi~wjhof1>r^|xN@rvkAtaY z5lGQDP=9i)CAEB1N1K71Iw1K5TYnq> zJ0YxT>$PgML9&POGd?M4dkDm1Odvy%M9rI|-SD{`Zd}$`LUae)Y)pix;3h>-8^3nV zeTXfg2CDg-m>}1E6!|b75X4)|(i|@}WuoAsiJpgLg)PlYdzpdAxATn`IK_V1U?t{V z$asM{$-boEHf)@X`P4S)`caUM<82C3$UUfrd!1S`VTLpH+^6k&=epgM$0aL1&CN0$ z?kZ!}81usA#-mI$E}i=fF-CKZp_sH~r9rj2EFr4;D!Ghimh=(O9EGxoDgQ{df6Hu$Xl0SWvlg z?%j7w*y*y}PjuXl7?-AO%PI9O^a{bq3_3WT@5<4~+YGsD$}Yk#74|;mGTmZJ4FzXq zeE@=4rm#ih>}0o$IGSI~!}5J24|QS5qBLsE!S=|G9vHG!##GX)2RX;W3EoD-|l zo{x=rh}Gue(ra^H`|2{;YD=|=wWW^7=DO%=YcuKfwY4vZw)wlJ)5LnOGJ8b~FkMfp zS<7IjqP1<&s`s>Fu}iaA51sF{=wp2PSJ|TZFLruI4c^|H;Ho5gn{tZY$g!WIGMKwQ z*2b-G8h@tQD(8I2WO65R;m^jNEg~H<-E{G}{qbDE#{P75kcDx5>ogau(!785cSJji7oKUX2@&7VaF7ngX zjrGZFcdtpvnK?H+KBtqmMor56n`F;mKtF?1#vId^MFE`|wvV*!Fxt22D6um(!Win2 zV^~9zUMR4Frjyd-v z`MhD0l?xc3aotYNrWepwYM$`9WXo&47-x)`wtX!sMpmOTg>8umN=U=avT7AIQ1lF1 zTAC@@FS+>T*Rj6fOq_qVbfTPy=M0>5>X^t7Q*Z73q5{=K_P^U%J9U_W3~|!igf}5* zYKO>q5rlE(U+Oiy?RUObTD&X}_&mdSjFJ5zR%0*@zwu1UT4{a>nmtc+y*ku|3*0NC>_In7e+QM~m@?vKSq^^$cy_Zhx)T6o`6)uC%-r{$< zEFU<4c7=2=oj=vAfMIwTvG7iB+)G%(5qccotP0k1!*3|ysFKH!KF{UZT`SBH{LFzo zOB&%-AAsV>9jYC@i5B;0CdqfdWb!5LBuqHin|9g-{=L-wv=JRxO8rBB)~Y_coylSM zgueFeo+4ttMZaWVqGs#PNXcilZOdbAi126a=xZ81Jo$F~#`6%F`VF#bUGNy`7mLGp z&En-n&n)%yk&fH~cjLv?{2|Nx`SL{b!z{i{jAmZeYO_GpSVIP9D4BUI#}c#)&>frJfsVeh)^z18SHlDn1x6 zeosd}SJ2MrFx~{_Sny$f`0jwNjW;uIaCb66u0SqqpG5~$=rZqUEP?{|VDlBPUu7XwU>I8F z0c9-6g~Or#>cMy7!Hxioz_gHhf6cICfViP>&~uno8gbeRe)VEV&s~TJ3>(Y^dU~1k!vELe1}H8q zZ}^wREx)O(qU#@vTWe3tzrKBANBgF}EN%n87nWC6tC!X`x3=Rpx~N4(Gem_Ie@N|I zZ0>cyp@_*ICa7~^V}t#;{8+xWr>U#^smCut9fUpGTD#pFIt4aPIv12LCs!yQ7>IgJYv!--aB zJSiUo9{Yw}#S(H<5)>W>H=L=dY*}Z|cPC5AwIZG#j>UM9idcHCLFzNwxa?&#cC*v{ z=62U;SxL9k>vIc{5}Rqe2YHx=1g6zt`nwnh3*Yg4eNT26+$UlcjssbL7Oz)KF}Jg> z+jSCh{Fo+q3jKE@@w%CVgSAm%Xc~nTE^0B6f~yQ9fez&3!J(TT+UTxp&hUJ;L+>?q z6z(OK@H(%iR<9>)<}SBSvYhwl?j6`@&TqP1Z%^JUosQdtzAx4@KR4e!Qf$i(1}V(a z%t7If#RGv+GQzzx49r(jVoM|_JlN)90YEI0+*W)OHB5@E5 zL1dxWP?F`cwUVA+yvR1eKF%C<|pJwKU3x zpzSayEE}L+p#`Nt83t)u!VQLJP(~S{a$Kl~$nyB!FJ2aga8evEemSh`E^p~UpE@FZ zR+OwRMrV_{Y}iRi2ih!u*1D~lFGA&%=D&;zv7#_gOL&PgNL<(;rC&)lUT$e$GJ;bZ zlRX4?*{h)42a9a76+DEI{;^Be3DyCBRS`4|e;#nZ=3e_b#4!GbRTZ}+Itr<>!*kZF zG9xsG2#Icu12#)N&ldKfAXp4TSUSU{np$m06Y8j#w;&1$?I~Ch#RZzH~*+KupAfK#(gbD zo9Pxr`~6QH3BhUAEk?&d z4DAJ}5JCW#!_S9Ak&lm0knmhmU=Z~8rYY0I#PoDzv2Z5kd^jv791yCo7ENs+GV@9x zQOZtmVYC(U3YzW&HE54rJ25O#Q63z>LmNuk1cMQ=NOgk05TYidWfE|+*Bk>9M#?@| zFiJG%bfktNNze@(Ih7=-g!AX|4j2u25R%(-wN^e;n7T1U?A8bq+T^YWILr$9F^E{v z-)kuSBGR@3!9n_tS!6z1!F_CypdU$bsub{}kOOMNm{D}mpqa|>&oB%WJvVWFTneX? z${^WuXf{)Ju6Df3oVrV57)y$0hHcOe z!;|I&2yEAOG~{d>2#;{Oe$V2AHGszBi3y0C5yB2h57Il7$`0K~5sMyWucL|7RC*Qh z?bA0X^N~8qD6jNqry3xjE#ds+uSio4V}D|Umzo&I^=T2~XmCX@l4G&boOo3@ zoVq1Z{Z6e=POH+5;8MZ*-5j!2GpTJ)+DxeURObP(I53xlZud7-kD0QqAi=WE#*scc zDqg)nz0%SU(WxHQV@v&91qWWGoWZcU>2AY1GBc8 zBIXP^LoE;&Qc=0g-#%1&V-NSK4JcdVn3}1LZam)76>U2$>@0us!JbD6H&4?tv3cqF z)VYIL>pm*2i}`%ikSD8}l3<+MASo?Sg{{?6w;Or8_SEw{b>&KD4}RQJ!N5R4pQ#P1 zHQOc9`6^onK(mJbG{oX^%BE+uuzl0F+2})^oT>}qoZQ_n5X7kP8Tw@s83{(zQao^7 zfwvuKXd!_ewqV1|`sF?SZzk0LJBJk@UuPTtyTc8A5M3~#|EWj+yJF<^GjjtZP)4jHP@aOR8_#|lm^q;%^o3``Y zhsUSqm#F*cp`^iAFgSb(d6XZjAQS}jz`A@97kDN9z(aG=;n1(U{S8d!%;5+k&Io)- zR2=?DYNcDHc1Gi|R3@Ww!_o%RDIpFg%A~Qx*(6-SKsW*^XQ|(5Y%X6tdVwe(Qoc|s zTT0~=%9^kUVRgK@a=Av45C1zlXy9t2`Fuq*)1%2;iT$U|aJPg+^=7B}sxFxVCgT2igay(?m;YtUjAr{~haZK7#!LZ!k7!Arx=Y06pC$!szB0FUlW-{DaZ4O;Li>Kg{NsQDE@$LEI5;bnlLa;C(R)2DM~$` zBBK33&jv)%X*>GL>1g}ri%T9Mr6y(zi2mgA~1%P#?J%3nsfswIL`J`j3j)E=nDmfUiUWU;SP zk5oQxdm6^19Kj_el-h>Zlq?PmT%|HDQmJJ@k))0XBtEZ)C8>0}xJo6ivK=%$;JzKx zg_)ui70k()m=vx`H4(om&EZfjX+l3napOKP#`Wp~PT_tBNmXm?&@?3AZdM=vsj%js zTuPH_qLs6vl_?{2ujcEDnWv-Oki%q1-0o?zjo2Siy431TX`>?Rui!Y(9r*~e?mBRV zv+RCYd$T4zfqSYcx=3lVPX%G5vnNqYcs~7<4I!72t?px&nHx+Hqrd5nGW2G4N9T2E zsKixjWojnWq+=09dt0MUx#ysh=qaYao66&ST*s^z`h7la3rRRhCG7w^*T9A!xSBU9 zmb<^&e|I{mF7Znot=X9{7UH%_MQc7@x~`@xsg&1cb8@*nWKJE4F|`8KSGyqFlh^i`+XY9! zz&lSiZe_c_4=ZMVe<2Pe>xZ9qLK%GxpB4FP;#KvEb9>Nd{Q^DSu1Bw?-foxedfD!` zYd=4R4}O1sf@FzLfl2g%-46Ic$>f3Y2lPSmTnAW;XF^cH^uy^|0kB20AUV^@5Ip6X z@ey|81SEt@!mfi*X!GIhF#l~V{*U1Ke_xJlzwW=N{}1H|0QD~nF$Mo$;^zW#DAICL z)^F9-sde$F2?PX*`1}a;>bh@bkQEA;fI2=gy*k4{Mj%x60y*m^c z?Zj&7Y8ya;%_?=HKRD6{xba2A8>-g&;fKlTF0njICY{O%jmifj42I|qixuIHtROF- zO-c6wkKJf4o_-29*Y;w&8_<#87!6_Hqj}Q?R8Gms2eY`b9uUFyMJy&FlwN|U&DQ!ul8epr(@8V}Rf)&#)Z0NGo z>g3C>Z>V`f>)r6}Vv@d8-xIW8cdTZYA>euH!ddhAa`^o&3)NNdXP1~_!Nh63EF`Rx z64f$JzmK+6;xyato~pPjLXZBsyS&J3KHCMa9%YcgEA0YrBRf?^NqVz88yMDe)7PeF zLK(jkH=HIoLq1suza@}(B*-IBDgVnFB*h2`jpnW&Kf2)1TLeApd6CZ+gAOKyy&ZVB zXolzbnhb>T+X(~LQPpz6kdkIX;o37vEe_T&1?_KMiE(5&R))G#NHGDS=BC0NM{*=K^u3S=3hTDL~SR3}l!E=Vy< zCm1+N$dHmPLr(U_1dyiFeJmTgFo&=gq)~CFCb+ql!0u)IQ3xZ^`#=rL0xjq3H~jTV|o%zY!?Y&Fa%M&|Z-Kd~5PH8GxT&D}^#T6F?jcSTHtK37#NQzIqQW5gtN z-}QGx1RMWEi5i)nb(Na?F0P^8rg|r~)%WC!nSS->MJ~TC36nB=QSTg<#yN z423>BKdGZ@w6<^p_C!?-q5b9~YYHh3-q+gua$?7=E+MOT%u>SI8frXAFf1QZRuiol zwhl9dYK0t{Sg;$d=>&2g5#*2B#5|Y$?j$kim8aOIKsMw2auf0O2P|?2Hvjbh;vuWuj z4Ng!iTKc2#<_ZMnDclu_3!`w&{$+bpunu#T z+_&Zc7mReS@z^#Hyq193O9}yIwZpCD%EK{YQ!^D~pE85K!Y&d<8=c&YVhlavM)(mt11CI0 zOsL{KB}ADZ4;hyG=Riz+(%Bvpx9TL*scmrdI~gmuMu(g*YlgIw1~r-o0g9oLYfO_7 z`HlIcT}-@N+PTU{9~xh9Hd=P4<^b4DUFiG(E?V&MP>n&YR{eqQDsD!L-v}V0 zV{#6|i<^J${WhC?q2%q7IuYvd((cLFKmZmR=!-O0Fbq4{d>-Xvv;qocl1K=5Gxp?s=X z99|1DfDikWz&|Ct`&@t09M8^E0SGoiSkc_TpCOgpQQpCoP?==0njv zJgvZ+h>-6HQ@K}mkPM{ylq&;jsV{y6R^{uWB$OakGTR8n-Ku)w|5GFLWA^^n?29?|E5RVi_g{P8vW&{oqPp z4)5lk*7DAQ!HAxblD^R<@1dx%@!#V>zuKv#`tG^%+2-Y~)U}D#-O7!{fy3sLk&W%k z`2B;krmKhPoadKJ=JeN(+`qT%`%jES{eV{_&xTPZ1;HDCqh30^cBL=-Y`ki9n=2FGeJQY1i7}+Ab9^e2nLSXzs|9TSzU5~)4BCcFw zm>Q)cj;W*JYWjPN@oqR;>(OMThQN(~Q>)o(rNLr-gm1gSv?t4C(AM|0opO5@;%LSG zF+hI90fxt4+Yx~l8q60oY-GzaR>FzCNnmm)}24Z!;YmW`>DJ|&{%?Vsfg`w^PbOsdH-N*njR6HpsM_v@4# z^F8jPOjjh$(rm9;VhO}RcSu!wkMQKfTyNEkqhb%m2%4Ov8XR@C1!RtFE*#BP16M*R zsDd!=0(ZlR$CEPBAUcLroIoIGo>i_GsoR+w$JVl8WdmTTAf_IY7zi5ZZZ1eafT*z( z07u!TpD?Dde$ax1eq?N$$p7dc9I~pYV_0fApYK`0f>+=+G%{}3^@7ofQ%$=zUK9^^ zc`8(k#Dh)d!&N{{*F#I$Y2A3r4=>#`HX`voVbL(t<})7Qva`08gJTr0op|1;f)C$v z+14#*_HDKDcEFNx;F=-Ssu$$iGpkkx%qz40p~d^Uo{9E7w9}6Z{=$DwD#~FPU2EfF zZo^jlpcN%#LMuhpfB+>$8wuQ0So@63RNd$=C{>18&8n$hDCTKPdeBIFd9lx4a zIL_0$ed~9#f=G_nMf5mVm8BxLd4A54jOop1y<7o27J24;X_ks}$0yrysHaKC-vYe| z&OI&DRaR4(Z(3(B9PBS1USUGolpZt#riD`{{hM12o0l$aVw{TZei2Fe3{CM3 zp@}U*XTq?zF!#Oeuzt@X8clCrY9(#97z0pEYebf~y3oSGQGUVbwV#>om{!t$AqnOp zjgQP(%-MSqCl?I1L4-{KBaVWE883ydN1j(s^kMIig<7!R1jAhqW9s<5JnM6gA{xXO zr~~1g4hkRE2QgqCuwj*VO2}2GF3IGKE}CM3NeT0?8{^Xvs5hL2Ik2;8=dM6Yh#f!o zL-)PbxEzKHW!FFi=?AmbT8iy#Z+p8;h|Jaj*KaR z-9ZOzjd>ah;E={9b= zlz;^iYCK`@rU-vC3bJ@i(US6_c==N%PNo@WLS*{39M*r+SQ{b0ky&8K+L~*X5&TcO#zih7XaiM{3T$8fKXsFzHuV@?KnH4*H zjGyKu8~4^C+-Xp0v!%FXKBAs0SyOSwP(FdQ%0Db)&ir^}exmATU_U zul0d}qpz|CEH+i_r>wpE8kY>$fJ*s!@-@pC^x|`dW6$^QQ)h1c-ZQE1ogAB&en;nU zpl+O9U3}Methh4I9PWC0A=lcQ(uW$m?3}m7v`~eWhfk0knSqcFet!^5K-O1F;|{6= z<~xp>U0*$+{pWne2pUtRt~k&bntVm0b5Nx$gq5Z4pa`O6|BA`#@!>{0sj?bk_*n!; z$<`aGMWrw&+bD!uHR$eXu(UPJs9~8~Ok_N|wzWh}t?o^t>MGLxFp)5H421Hw(&sq>onS)XKv&D z_bFjCi0c<+M)0rS=Ks=fhWq_zUXou#L~x3C(!cVOa?*oB(*J`ptBm%osR7jfqfM#v zZfUP-Z>sNV>1nJQ?Ck3AZ65lf%*q-^fxTaeNo@=5^Ruf9Q8iP;D@)4_>$TfQ$I}OU z`+Z;j&Fk)~g+IW@rjyvk+t&A&SFjS1Aju>lKoQc{&$zsDe+U8zp(nM2e7-3ZjnqO~ zSRrWuaVUS`^6zf~?o?Ot${H-i%(9aoV}nO5MY1ln+Ll_Y2}kVNXJ4gYQHD|~moh%1 zb8KNAnExwZi~x?Ubk1HL@(aOEvT7>P=On308za+*g#CTXoXLD4){61T$YNEQVj~)8 za3_^P+e|xArFP_UQGOnb$>l`(Ig*=}cT(DAO_#YomAL|=)tFv5f_n`V( zwe3<=MqoYXRot>+;2D7WuFd>VIBI(BUFHBVm!^fE44;I1ah^{X|wV$-;0} zD2pN`q4bobl!IhfWskCKvlZV{9Q$=!un#OE>pL7ZqQHek5VOD-ZL`wAr*sQ5-v{n6 z{LK7ejtytGC3h-U_cgOzBRk<^bB8JZ>rl5chm+XU3x78ngCS+gY**_HNz;tDUCP`b zTU5>ctgs~y^W24#@S`dpfTmX71Ww3$e#jPVX~8!ZU|HdiSXhZclCMR1iH0eUZsVcL ztEOju&gzy|Nys=}oqf&e8E6e*29dAV^cYp$=`Ig-?$aW%Z9zXFlXX4v+ER`Ij^T=d zHeS!qr5=-qb>y(|FmsAHI?9a#P{t8;lW5MpG9xlRDSD&NL@e>u!eJFs4P!ELRT+v{ ztaC}quZmO(1GRm7$=%Q5TJX@EykTJD#Sx@@HeXSb!_mF}o@_jRqhs*W<6A@90+ z!ZPb>3HPe&e$Uj-7C0cY*tn}wQMH_fv`>@H`{Z5y+s1L7p%MX8$2RA1^>0H&f77OpjO*(LQRgRZFj;kn_hJzbZY!_Wr4%CB?Dkd_tKU$zY46ZGpuRa zaIW;??352hIzNv7@M$&$6l&;oLHAS>)R<}w;;c?SwLz6II+hy}t@l0js71qDWMpPe z+b*{2V8A(hp=}mOsPXjWUwZAQX?$01qZlHQ_pT%++Fx?&QOrkFv5cIDAPFeb`D-oI zH2cjf1lTUoPZ=g0x%MHp(pw=!t5`9Q9Ku7|x%#&<0(2hh;?S@&Fk8ZnX`JBd5=;!s z;I|Ypo#G=Hfb>+QRD211E>1OIU{Spy7|PiI4pt&5&Hax*;ALEvZ#b<(mYUopYmz_% zhS0QJp3av(j}WaqRjS3%%b#AG^0Q&gDyxvt%JrR$kzR6~dc+pxRXyxXS$tfjD@HHQ zx|seK%D+t@B|*(V;6cUz$4O%p=NFzJL!X5whK;StC5xgCm;9d*bIF_-nOV(Q8o7BL z*jikexYTw35iu8*+SD?c)A_rLOSlyHwfIlbTyrg47!mc!)iz*EsGm#MdVx=0bwC!f>S2QHD{tYy-%4S;{i2c@;z8!JcCCM=&f`?0;et`W=Vn5&R#TZI{Q)_h)r&d-1oqp$xr_>^!$2chc0&|yN zuKRVfQQdy5^=kc~%K4ppqi3fc)YaAg1_ejDkYcf$hQ8rI4MglH^@j|@iSH*_ z8^$(5UYx{AXM6n-WgGSe8xi+sQo$De4o=tro3 z5Q7+^WQpg8VgXk3!|)B&DSdT)B}+2QY}KPYJzqjhA_gCn_0p_TXiS4WPFIh@ETGzs z{QN?m379>igZHv7EoMaZ@OM2*(s41z4~qz(I=1tkHKdO7zkive3Zgi$gq%$wjbY;1 zKitU^SWZ{=D>>wFC@NbI$Hft3kWh=O+rhl(l48PnP|<3V*Yu&dfWZkxX@Db|(qu8M z{F24jRny!sTrZUJaj?>2Z3f}Ule`67%W4lY&$W}5Jh;R)t(_3~FHw@I{$ej*g$!t&t&Q>P1Kc@&+y+^oz ziDv#te!rMsK{0A}abb#QvzL}$yq`I{wZ2$9Q~X=rVr|AR11h0!Ie@PB$-ywlhzX#p zuJz0c8^L*so*c#dy+RuU-5*T@&bNS=iHtHWy68YGck!qw$mD;5D#6=hs8^xL@Jh^l zj&iXPvu&e_h8%61h3_KT8HDYk6LG@fG zs8@{Rr+u=-pa59jl`{Yk=^qfMs?8}Oy?w4q=jmEvm8iLe!j+M_XgwF`)eo)uGRccJ zAS#lG-7fL>^&?C48ed8;`LL18CzFFDvU;*6*iFN`Pn2hyZ6av5^$Okbv z=fhB}3_?DDhp?mtxucqx34OP=SuiSQB*e&{KH{5kl0s z1|{-f8763P+;8!ViABCCeG<}Ac76~pGezzj?)enRdGR@HIruPry+RQQyb{y{h>Ebi zcvVcXwO!MM=YkXHH4Vw2q>;V{S;CxPOlb7R#!&ECqsA7Bm}&!qc$9C`o$x`4 zu!(^d4E?@kLnS2Q_K;{GPLaPB^ic+AJf)mT__j1Ti3n2cQ2WL9-m%6 zLBQee(ev~A1HTM93ruke-w@%*7|v(w_K3nzS;h3t>kCI?_g=HXCChRo5S}Ft_uWv7 z$G?`{x>m3oYmC6fQW@*OlPlPI?S>|CU z zeC^4M?tD62xrsSlDp_tV*(HmnORef)u}Ub&^yBExQnEZrt+d*XB)D;kn=MY=>N)9f zzdarUa|>@Zxw#LHLks=ocjCE4WL4X(LlpmR{ro_+Bfc5mlIBqu&6!66Q}MV7K;}^1 zv4nOl)ei!&eVudSb2#VPd)$QX`mI2$?1rCuujKq96_Y4XpdMM74hmNZ+YH9GJYEfC zc~nP?4!T)UaG%=6&X384J=ydJ(2LYyI9v3i(YX)L*(N-fz=oZG)kQ%2(n_)1aqMS`LB3go~Y z&$QwoHv>2ehZUK`^Hk7~#${+27sKnGZj=<+Dw_LxyJ~C~P^O&HD)cpL&gKtM&=ObH z9*)zbr^B(G#N;7SpBibkfb&1z-Z61>u2B`E5w_?%qvsP#&B&4wVcl`T6aR; zE?Ai)1;L1R%^tV9rRO{P+YX0pwee3z{1k)%ztB=ou z_1zTLI^W&2>fx8Ap7V~LEkqE@=6*(E-uiy-C)LHhql~!E{k(E3<-?+^TJ?jTx*OvC zl78!-hZTWP8xCdT(&)QYYw+U7b)MGmoRap-o1+`@?qJ9;&?M92sP3}iZ+qTmt(fPg{mKf`DALaS&wIAlE_KY3pX4ZI;mL>MQo&`7M z@SN98AaP$bulw>|R-E~9yHvFz_*6F+|Z4P5tQ|9CkgV*do(ZGVCs zbArl6`}?r!FCnLKLi$DrAUf->5@QR&jzk088}(Px30!Ij9qVTHz;{@~yf(=;A;5Ig)o1^bC z4ZeF8L?yGaT9ej?N5GQ))MjF6=P~U~xmp0*2da4YkK}@*b!^Jk^Dc7-p_FS>a@sND zF6&sPlxKNt+O^{@`#P$W?})n**19j}xvNy*ZtS-YDhVYOvG5P*j^AFWyQu`H{ScU6 zv7$Wp`G%rLu@!w6% zBtTi_1jfGN*Ds1vXvgytL!3%JzrNJdu%%o|*yzK9XjHUm2Mb;6vji7I<)u6Da{;Us zFSKc=vYUxY0+jJTwH{!3zY8rj=#o;#9T#i*q=yG3>{Q!FS85;O&N@MX{r_RH7{8!} z|4mzoV*4kpIw9si)2d@QIPmid^8b<67LezBNo!HjX#XXxMMM48@vY{ot)!yiHw@Aa zj}DE|R%W*F{*zWciv~ryOv>7~9#+>l|AjH^?fyn1gg$}p?D#tF%dV&0xnE#kPM@CR ze*gRN7tI_38+!+&#j0!P;g-^ByVnoz!#@p1IJ+MQyP0MKSRcn{{!Ii7U7abr7?%1g zJ+&S5Pe>}0-Ul~HehGvWJHqk8s=|CYAzffg6wO`iRAz!D^;G-3;P29Pf2eH1rV5!1 z2@86YBF42rNc};El^sxj4G^Tc3JQs9eU7q-PweTt(_i!}VWu{DDrf$rVF@QJmO;|$ z^P>)b2arHJ+iq4Gya#!YBZ8e)D#&?*07cZ>aJ=V>d=r_tWB1cg= zzliKMYajanE|s5yc<+FbzE)tWFaW&#)bb~lACrrvbcO2FLBBr)ruj~QxTO66Y$APe zlJJj*)H!)kSjU}U2&tWZ1$eQ&Jd3kz%4B&JNrSy`zk9)f2s%>r#84^bY#3B0;T`{tOiV~FvUHzJe?JUL_JdUB-D#YsvW3))B-kfFsX+FPM|n!gBV z`U|35cyk=5}7K)?i?J7`J;yx}U!ib(B;P*{pGPT{H=DrQiS*ng^H&*5qf4Qv|5T_cBy@=xI|+KN)V7%D1Tq^#@h+as-85^N{V zivbWf;#IR-Z{`-wS^V_k!O>i4j>iItUaZ+|QRdk-OwE8_C`|Yn8HkT{mAv zz0L7CR#BOV_$iS;z!USoeVY;FbJ?BLS_huxwv#{o z7_XmjOuNHf;o@Zc?Q?BiKt=*`^C;++cWF~$wM2wI!qop$&SpL{Lw#Ji;fz`K`x?07 zRfuTQ73>Xz5@68Zo1cGgdEfZ=w=S=`+rI3c(v!9BQ3Ah^4`v$(t4F7C2uaCdhI1b4S2Tkh@L znYPm(+L=!Ohwqzt-}jvJoDgBVJS-}UT3}RziTPwES5t;GeTOyK&yrQ_eH}2f)?9qo zWJy8Lh)9+DO@d69?ne+OXnA!SXsJkhm5&l@b*Sh^Lbpwxqa_n+hDYqvtVV@~sFXA3 zm`cX8yElATFHxzc5p6)U0T37sk-K$mliZ-!ZJF?GG)Y-MHA58Un6!>kOAqDA>>|~k z)ktQ|fDtKzQ72Z>AFE1F31dQ1Jr#|4?9X6FWWW+!pTmK3&Xt0vB5@3x_NHyfIy>Ly zli&m2{78tW=q&m6xkzT1XC^@pnq!n1bUMh7*}kUNH6rMgh8)OhPr(d%of=h&vTNBC4T zQln;I&Zm8dgjS|iA^k73y?0eQ7umcyDkEM`fTiz*c1`>>DMl9yX7}|rtYOQbpI~5_q9yFf4_C-Q6bmLOKbtzH)eK?vUXNDG z!m0TXc2?NJ1Q6{-79psYA<9De~BnG=Q) zSuH|qBi2U{66E7sh3ZO}23fjy-xHODjzzH2L#6)T zQ=OTPg**ydT`E!LlR{p=t2hqwHSiqI`hoP9__=zh|8l2v=USa9rT$M}B9a@SOEn1cXc3L_7GjLTT z#vnDVOeAA)?qV%!sb`SCrD3Qap4}ntXv_7^LRydfivNUV8XLK#m0k;^lhEa{(j+iS z7jx@YXRYkE@jTwSa--$Hvm}bKHYf$U)G@@ETCvpT-bmZKy>NKDMp7;qCy@FY`q3*L z@2kp9q${|LW{N~z!4h#xnDgO&xa&ZCH4tLdVQzNwpkf1A&SXYj>u|AXe>Mr6|g zUh`?o07nFu@HcF z8=y%;4b}EeU!jhR3`9H*&{+-C&-CJJ@X~9*r#5l;ArQe49E8H;gKia3Mnu;{z}1`K zQ^4tG_lt1y5baV@*i`J>?P9Q~D|YBjm_Vklkj{6JV*}6U(AlA|0Dhb`4{RkdyDivI z(qmE?e9}NwG8Hw_KoydUM?ZZYg!?`Ne9mavdbHME^v{pzcZH;QLG~)N#N5}BFm$jV z3eg*gWALt#32BJCyd2xKF=>Bc!E$1+*2j#LyN7ckVyeY0Tg9nU$E`!+Hdo`epW}9k zGviMo@#m}Ym(TIn#0l^#`1cSzbq>UbReZj*ghyb)gMjN21MwOm zOqG`Bpk){g9Wkvj4uWoC*J0R0D;nC1`;_FnzK5B)YlrsY(fnvDd>bW6%rm!&Q~&4= z86`9Y6H-}!X+d_x^~OL*N+xYg#)p`Pi`p4_VmJ|b@+J|oR`^Ja`p})A*?VGtou})u z@bI;gWU~&vAhdPT_K@F${g8;-)%F=JGYxIk+W^I9Kp<96C&En!&kE>m1i*5E1nD)V z8=?3x)274Ehg($O+gdrm7pGPAqDNrkNG7J2J_RKahvERVMCGEyb+yDE^lf+pKI%dp z%8A40o`CasnQ{P%KE|+!wGhAyTCfiNUY}v?NosUC>1A+$8YDmuF^Ulw$?TOC?#;FC znJBS_lJ_EAz#I^*&9ad7}tTx`vy6Kgf`>xiP~7e3r~&aA9BX1V9?QW z#Gj`Kr}7}Q=L}brU?I9-R^rrb!JJ%`oM@EncINE(cHsE}P*?i>vl%7>eYT^r>PeDq znzMpNqj+3$T-@?mQXCT2ay}gASDu81yikhie6iH_>?(=bYe)GVr497SSBJJj=SHaNsK}ikrj;$n!(}TBoFJ+v|5W&u)#WEwH`Vx6)V~|e!&;kr z0&3s=d09V)!HsQ$9b+Rc%@cF;k=@gkV+(8j%RN&~>jm4(dpo;_^-G(hCjm#7ztV4R zQxflPu3{b@Z(rZio?*D9Zbuvo%Ypzn6**Gz`u!2Z)Itif&e*~ZxSE;>kVH-C*sCvG zUBHHQX}=G7QHx%<1|yoR2~iV-cY5+^%=+_wDCdWt^5y}rXN+!?bD;zAsG54ISPRCY z86${Y*gzC*#)pOiC2* z1fOqW%=niV@GS_9_sJ+2%_X_e^;82g<`Z_nzwSFQ3f}U*x6X`;jEeHgjLLF)&WL{I zB&3ZlofFXYX3K|A#)cfM0z(bF5sQRymk;)7iIgC-&?Hbj{STdka~J(2LCy-<St!`lQXdzF z68>V$&W4QT7YCvY92G=9dSfQKG4q?rola8sgJ_9W24D+u;ZDnuojOj?vEFw!L99&q zQ&`wRQdSI}M3#PZ$aK($kj0Dsld6*C>xvrMav_Yeu0sa%5G(8P3UCQ3?O8J{jIB*W z)xVn{o|=F1EG^ASZ#K0M(vXAjelN7DYU(gFI6y(hd||w(1(cEaY*$P;GG98vBX@z#sRp+iVh+xu*7zMzl7bhe%f2R z`WYyC&NUb+M1DKi!KQdT{Aoah2@$`5`OuqKz3`?N)7P$f8oBPkEE(Yoq&g@o8`Zu% zP=&0wbD=+{rTUlJ8%t}Y!<$@NPLF&I0&acP!;=1lPf&ef@*9j&_i1tMx|MJtKaB6$ zMeqBoi?*>>cw_v%fFpl&4pOJ0{@9|^geJJcQ*H0w)tX#Y#b6d={Mc+&_0hXJscGwB zCD$Xgw$3~>x(y-V?jq9Y^w`>Zj=3O=!m$Q zLiP8Qf9#(ihwqm+!zpxt90L0_G)0<~6h7*(?v4U3(LNbxJb*7djlMAn5LQM9Weio) zx-$J2YEI8Ar;vyu?~AZmI*WPyRfSYNl`=(a#O0+|f{oJ|rji;2X@DrnIfrUY?yXye zOYwk`T&xi`J8h6**1db-gBEg@jjg2FJ??T07mCmRsjw)u;KbMHuMh__4F;^=#AEJ) zg<_b3d-FIVL%l$VB`1;$dL`L$ImO3a?Niul_c>ZIM+bu_a4yrexks@SJHt^e^L=Nr zr2|ZWpY7~oR@YcFTrg~VkQu>OS0D^V`8VWO{_cZs37J)v4d{LH>?mAhdoL1k#{1(0 z#Be}o8C&6`UDb>VC>P#=Tz&}3@LZx(0WSah83;`O!3+h z0oEUDm0CB>=?$^rtnA2d3Q55s0ud#~3Y5`(RzF;enF(n4TPo(Qtru;`AFKpnvFE$v znoH>8Xym_^OIiO7jkAL$l>!&jpGC9JsEbaAGW_vZ>sr5kW%D*C0 z=`myxo;KXk>F7=AjaEBs4sWjA$g|ZHvwOJ55Ywv zGN+Jvi)Opy#ZaST3;L&&@!34xMU#Rp!oyIH=*hCFWCRhU(2^^19ypw}?d9L5ZJpMB5_LO`Ul!z+&AXB<0nrV8aJv^Da$N zeSs|gX^skhXqTOK$@4%i`c1Wj?%BY0M}vm{W9`YNp6~$SOzHx+&w@h#1KjJlE`xPZ zc#M@J%j?94MMf}%zr@j_O2NMt=AbvhErq-LDc{(;)+tp>4wvh$$ae@4(dXf>Xq>Yr zNS7V$xmLJbU3zhZHr*3`j>lTB1)ld$?!0-1!B_@lMgJ&cL#IZK@X#BUhRZ5R+Ff8C zbyiA&?t0)0PFf@373ME5!f;C_6ip)O1MHb|@U4oo(+2e&c8^n`m}gedhx&y_?~&Nt zi!2lhCal@bZVZQQrWKF5+hKtcF#xOoxbOv|eOt;el>u!^s57h)R(R;jctGQ&|kO zAdD_+XqWHnnK71r--z(##}HkEL`&fYMFJ0P3eq;qI$sR#o*nDoI46xC}I1qu24U=1^QoFbG$91**=hc3WEi4gpc zXe2kWJ0~k=|AW&m+Rn-Km7XiNv3M-l!Oep)7vB%pk8n(ZzNOBItvHV>0s*ao z^qxNdjsi`P*-m9x0TAZSM3yClz!c;F9hwklN%%4%C*sUtb|9d4#Aooxw;kT^2s^Zw z2X~$7T~!O+`{FJ0s%zJu&>;*U|0nq3OHhWg_xwUGFz#}S-7|X zW7^BbA#~rZSG;aA0$ALQ_J@KqET#NFQ14A*jfqT*%DvcYMkQN5uToEAHUSOm%usdF z&=yA6hRB3^a6*@i@8na$WSie4*dKuz_Gr)_NjGtfF%hOc@!zOlrwn`#vcEbE8mFv3 zKsS-p+Mh%>k>tdmmpIrG;5 zJyyGSi%+t!;9W|CC3F!ZNxv3=rVoh|Ai`!P31STz6FeGc-H0aCFan-N6Ie168lMs| z1hfA>Cy<@MTDc}N8vB<=C2SofmPYxZXZf`=0(dz9_|{3ojBIbf#G18ivazHpl3Xe| zR#7H()FngJd6ikoZ!*lP=J@6+>B)R2=3}x!j?An@czJ^@8TXobBtue0bo@fiL?X>; zDImwR$22>T(iJYd*NXH;TJCq$EDO2IgjhBIG4;jG{_5e)LW zdF#B)Z59i8#|Nxv$CD^%=B2d{m(-_1>a6sjEU6z^X@Ue|@}>U@v--sWJjBTYOv!67 zehTY8gDa&mxP(cF~yn(UNs+5YCSR zNKipqW3FpDh@!#XDVy9>2uzjDHq&71C0}B7T9V;Zmdso#7hL`ZQ<@P~4nI;ZK~wQq z2o6ulotVqTGXu-*mITLDX7~g>2WoIhd6WqrfdW`nWiFi)>-0< zj_*i&G!{$=1P&UXxgSHt1XqIXG3^z*>g{>2RVEZ1^2C^areSm9K@h zsm(I3=w>%Z`L+^e&)I}73Wc(w`8K_TPCxk%XeV7IHsZAV540x@ zF((OaME2^%mb&=vWhS$p1mXDx?PR8a1~PU}ytH3}gh?=-P>Wu`nZ|-Hae{SbV-fq> z0ei1z|0+`BA_ESF#p{QZc;s5Nn3a3Ilm~-q{j}i&ki8ZkTCPR|krP>M0rhd*K>(j* zXKN4F3TPZ)z1S?Q(D^A(>%^m!mVeGTFn@r(Ii!SIMkU}ae{aotGCD$bylm+xEB_^H zoH3ghEoYddte(slJ*orh)zqL1&pldt$%3Ijp0jk6Q$dm|TIJuuA6h9B`C+VJk;K7e zObxDFkUB>hiA6eMI4^0#97~7AKegy^Hs$msMbKQ5o)xfp7BMDVJQ0%&%;{cf6l|0Q zu$o@mEgXCI{(kSmQm^0l<7>hZ`B{N0Fh6O(%P*1AI&4Q(51Jo$?# zK!NbNsl0A+OdlXLI)&CN#W2y5@Ez3nxg|HxJrQ05Bu7>xYe``!YvyfnvdKox0}zv;jF@JlRN?e0lT(w}J@;ahP0|#-ir$;@r`;<2UzLq|BNlp9 zvZG`+LSgT|Ar~7=zjuDlX`u7^2z6VQ#kGsQA(7a=ra~k87IUSZaHDRCJY# z$Vlh#SPDv3I_hY}68PA-GFp*5M1HjT)w(E~DEJR*L&s=vG^ko+tVVu3(}(#%7i?`k z(fp?(+-EFa9y^)^eaHZ1j2ufDry`Vz$8O}OA}GOc%Eu4f)+JG^IJM#T1GLD1v_lGo`6IGkG5JT0Qxfd~ADj9ENiO zA`*6RKF!z$u8$tCi+<xZKLKhtof-*XCX|EmEj z#y378F){Ytfc0NWd454sNK#3B0Vtxh6frOvTw51akx<<9&cTJG1$1`$`E~X7_4l{O z42<-Q1Po1Pj(7hYpITVV?Hij}>kFA!Snpb0U)m2{`!)0X7DF(f1cw&vT4KlYtemDJO;@%GD6HpBkKRZW<9c-r` zEAiTd{qJksFMpm^z6Xg-SRk^#*lCQLwdq|7U0JV9#V3DJBGPrSn#@)W96}Yk8?>m6 zY5gLct!MPp$r+8cI;CN+w?|O#{34@vdmZAie~6*hMs>Iw0{#BS#z$d)UX?abbKT8$ zt#Tj)N=}(yvzjqCvN3I__Ob+`^9S<>YWQnj#r3NxJ9&6NF7X~3xWS{Alex8j^q{7K zZB5GyU8aEFWgaJZ+I1(wMa~UJOrpYztY3XXkQl}iqW zrzNJ3(236~jT~Ugrp_=YwHZr`XXr35&9Ol#4o|#XJ}J#bwXC4`>hRvrm%yxK(Q>+d zv8htbJC%#CM$NYLX+jOhuBk=R1=qAq^0H(VaGlbZw;R81z|Ts*iT)T|kEjMsGqY5H zjk0M!HgkuOUBZWbc(aEf5ntEvEc&t9w?sVCzQ+ZYg&om3SKkd-&zENtG=BGIbwA;- z6rIr9N~`Mo)fIn;(jZOfJ9VG%b=5;*Un^Z3UVyM`MvXb%h}l5sg!ALTIcO)8Qd8QI=Lp>&p~1QC$PSo1j=vvQLiVNGE*@E5mF%vB;P-Q`hslMzZm5VtYwM2*v63&TjD03w zjzj#u-{3$#Pkrrm{@>z;=I z?c~4v8~XZBVbY>~H+6mhzoD1%^q#x}EZcL1tT(sA?h%X{B*L)G=@4YGQxav%&+xw< zaBqGNqg;m}-fzQxN8&D^zYYiM7>#(tGSl^nC6#7kmVr)4;n;-9< zl1Oo3H!?^egyY%#Mve~libC9;vN=5~9AD>$JZY-fw@mD!bb?z^YuOB6HVR19Ek2|C zqp^(&-Ct6)>J#c$4nN_d+c1zF`a;?tjXPKj-!wXeO*k9!gKeLfNO$-d2P42Xm6ELQ zppoAJ$6!xLALa}oNPzkgnsAp>FgbiA!vd$~#faV!=iCK0P+{_rzd&b-I@@>=f$#%<_m3H3fZv4RfEeEKbXIcnsX z=y&O}40CkQ*U#P~CZr3QY=e5x-!H<}H|Np!OKMwXsFip{E#pZ&!kAyobrd(ZI^R7RDPc(>@NnGM{gk$%FKOHVt zue;;wN|eaxzHspN%-C@X@9*+>X*@GOB>UJ_KHRVK_2y3%X%JLv%DwB+v~cF!$5jBR zyxc#MZ+_!%l!z&(!)32(4YTvCR46}Mfzn!y^PQFEmhZh1acUV$@b;e)i2BoiH5P)t z*;%Sr)#6aXS!emxT$?E9t_!UY$f8#XuINom$nsZL+h#Iv>#iL6c#>u!vaQKl%?%NH zwq(PQN@u-oJ%R=d8y~2Ay6FKmLo40SQ*{AP6FdK2pMJt%)(0Ug?!u8h_X3paLx?7K zk%XW7u;c5Y?|!;psQSE$1GwxO`tge%QH)j1~BKWdq zvKcz8SLaP=R~x{Y9H(z1&xN_7-r~sHlS;1^Rc$t4LAYhNapod0-YOUnRGxG5^B}#G z_`cC#@Hwr{dq#izFv)l5*D#v+3iCM zkaZ7?YMC>zMm>@zL=Jf)qm)l;izsK%Eg$lDO`L6;85lFtBJ3(HJSTpp8xb1Rma%Z{ z0N(d0{adAduT95(%JC~W^Fp+O)ouP)wx7>Wl$*u7^jQZ#S4eMu5^QSpdHRX5eLgi- z8(Dgi`ch-0R3*ih$33%cz2ffBk@Cp0uHAOR3|UwReUw&2Z8-Q1_AaU1{(Fn`bh@+0 zHPuM?We@X{(QG&usPoa`WCR_QF8}nwZ?x&FqI94Vl5M?(tW|TMM63FEq&NbBp#M-H zyww*qJduYmn{f|yXMBO#<)BAPBT@)TTRI=1ju;#y7`9JQyje5LNn$GlQ{D_3aHMU> zA&V1~s%!YctwkzCFf;@-5?sabHFCl4b2(1Pe;}X&awN2n1#pD@K z?%JC~91nC(QEf1{KIw)>wm-*odqt0P;rc>c&-;xd*G6MN$3#35@h8V| zO)I!xapJ3q`J2n)hP`mPaAMsf$*;Z0F4JPii6c|h-DgLm=49xWiJWv8BMynVra}^2 zEt%dR0so{D3PJH0Ywz4i6X@r7@$l?W)m(s|-|YC};B@z7>t^-x-rl=@TJ^529c`ZZou2Qe_Uzo|uFow$ z&&MymzWp)#^=3Iq`N5|bT@6krmUPELwiz>9JMO5tnrQUw7GfEFtk5?i@&Y7IL2C&)^vj~S-E)MZX2Ea zZd18e%61r^ZS~r?Fh1f@QPH7su6rBzvHQ%uT4Tmp>}UAKwX$dz{n71S=gX|syo;14 zgW;>|c4E6u8~Z(}gWbB4c_{CeuZNsLhbQB!ElK&_h#(d}SOk;v;WUJr|GVX9=ci)9 zA17LgQlcqTIWb_u??fN3U0YXnS~qIa55;t=jeDs@C4B1p=Z3xq!NF-`I1dkjjm1ii z3Ht(wiIaj|fSC`JzJPw|T~QLa#qUmcoJ`T7*z&SPjA7lT(;gv1iCao56~uIk?CYg? z!ovQHTd`cU&RH{> zwc(&rT{<{#U?Ps5H~pvPxcmGJm0<6T9JJ_rlr-zwu{727n~E(O|2=@NU_lUn7ONL- z#ZlllXX#`hrGEtxvju4gs5s`d;#;X1sT^Zz>N=5Glno!3IJptbFG)~!tDbBz2~$xJ zE1ValNAe0a9ltEzF1wM2a@c9lqGPUp?j8$&qH$g9 z6Qw|0++goTuhX)o$qpl_?x|(oXk2H* zj(7WE;2}<8i^lw1E!3JuHTTqVl6?;_AHyy@XicBco0jCN*kspj+x|Xov;7hQPUHRj zb2k~jJLQfh#PgSD$6>_KV+)OJ+a32~b7C0g_)W=6I8wj1fwsE`sSP(37Rhao*^GVb zR-o*q_(8g9fErIhJc;FziUrAQO6$&fLmBlIo4|Y+-k#8T2ALWv-UB49&=G!bTQJvT z-14)334ej>bud|t(C_f7sR!s;8Y4~w4O4DE978GxGF}X`^wF1urEf=@rXkB#qklM0 zkMUdr@DO$-54-z*VA$`Sk=~lg04)COAz@c*(k?c2p3Z^TExk1mrNC$-{&ije1F{!3 z&mM7sL=*zqABfSs093UPtS!Q7ITnl+`)-4wNFp0sxB~sZvsC;Q3qw#DHj2sl+#gqbe;qqYv#T99ANIX9=&0~bAlkdP`IQO&F)&mVbWDsc+2~uh$ z#p;mLl5T|)%u+}w@ieC}E(5;ow>S8qlS`5uj8;wch4asp$m5pqR0tzpW5G-2ZMuwh z?5E8{hs2f;9?8;t9ioG(6o^d5INZ~EV58#yiHW?DMHX2=sS8qF{Bib=SXf}WKUQA@ z7#@T-u&AP(;GtMbJ8>@zbWRA;cq_!Wlp9i5$0dOL{*a?`JHfzy0!*skMf-+_=98%! z&O9jIT$%{WmZ=Vp3S)_QP91>@QAZbH3K8kNomJoDh1tpMHN=+;8C|b7rrvc&qeFq7ZXpFOY{4n5=!q0KBZ9ovyuE+xI4r7Bth@q zQ-W^fbYI31SE0D14Xe~4Bhj@_&d1*nw;5HIM3GaW5-G2t4^hc%TVTHMs+6q5W;9CY zDAxeXFIDP2R3eE=1+&CQf0N>CQ3aTnVk|J>lYRqpUYy~Wd(dP-lL!d$-OBkgDq|wq kZ)#@B_c^kBguR`PYrQbojPjUPbrAlKvJn4AQ9a@R1~`BM!vFvP literal 0 HcmV?d00001 diff --git a/docs/ja/images/logo.svg b/docs/ja/images/logo.svg new file mode 100644 index 00000000000..70662da887e --- /dev/null +++ b/docs/ja/images/logo.svg @@ -0,0 +1,12 @@ + + + + + + + + + diff --git a/docs/ja/images/row_oriented.gif b/docs/ja/images/row_oriented.gif new file mode 100644 index 0000000000000000000000000000000000000000..53daa20f322d37f67652e5f53c26a433ea8cb5c9 GIT binary patch literal 41571 zcmeFZbzD{3+BS?>NLolq2uO#ZbQmZdib{7k2uOD>Sg`2s?(S|_)S|n)yF=!;0QcT{ zp8Y)MJ?A^`AK&l!{6k^7<{0;!Yg~6+_caEe052<>vKb=cjYWF|pj!q|7uet52Xd7G zl$(IqZJ;Rxa2EvTB7uH?U_BP7LjZd10cJvA#2CnX2aq5FPi_HSY=9I4(CiL4H~{|I zz)?SNwgNCR03irARu-u#Nq`g&P=EmFssUZGz)&)fLI=dg$`_6RA&$T*6rdpiw51~> z!houHfSnFl&zzi`0B|1%cP#?R2mmt*@a7FrTMM-31B(uThbB%-2H33tj(PzE1fT-}fdAP{g#YynKoR}$VL`-zdw5l%{UJjp&-jA&1sgll zOZFFxL_hx!!)G&A(_j~W&--&O_-|aqTBfE(?DX{J=H_(fFX;@8HR&1H*x2Y_Fw!$J z(!xJMYhr0&s%$}PU_$c82=CQRRE@QbOtlRSh%QG|Rx$i;%0&#f^!gF>js6_gz=ZDV zap+VH_315?jp!NZUR*xXA0HGE`JX?kum9(#O-#kr|FZXQ2R8X)X{1gsu5M!Z-B?u} z{&AAasf^fpjMbG*4UNAT8tVP=7C&kkni`sD85$As@Lat$A_@^@Rc(XIip%ayyqf@k3*-et;Wv!{ypypwznTxc-}HGFnnNQV`O{D^YR5N+Y2VP7w>qO z{zDy}GfsnYtRUv7tWE)q}HZ|BWp?jPGB) zV`gE3kN4v5ZMnWz?Qd-Pc`x&SdoMlQ82Zbv_m{8xk4x}1bot|-ix>XOpUYp}0KVpp z;fwnQaPi~(?DXXL=nnGZgysRYI0(HY;Mw4sIRN7sjjN5C@(85DK083$j{5o$mNlr>kh>weniH?el z2oDPl2@VPj@b~lefp~j?Jv~6~ZmurQPL2-tcD6RwR+bj#X5URsjK3Kf8tCik>S$|e zYN)HJswgWdD#**p%1BE|N__qDS^U#SF;Nj=AwdCtKHd-SdEUK!^O~EBlY^a&m4%t< z)k{W(7xZ+rG}Kg-6y#*jNlA#G5fKvLKgGkv!Nz)m`S{Vp2lp}1(NIy4?;+j2bNkj! zM7WE1c5>_XT?Bgs8o0MWxOC(QcL4`mCubK|H+PVSC)mpy;^XV*9}pN6918=)S zwSrF{+0bdyJ)V#6 zP-!sYSYHh}V0E&C^2A@HEz2BWE+pm9Y-soWT7fQpR+Ka4=ROX4Bv1f*eM4_?SUEvL z<7t)?vs~ny+x>_p%E)e0#uP|EB%XM`<#(E4N3yZA6jypx~_)a%te{{(&;zn2AC9>(Au9^2dMR z$$!s}ckr`1;WzC7QQfC%cRp$9px^c3e0R7L8=lj&;vy9EWT6E0@UW##&Fq{zJh38548I}04ASnwsQ}Rm`@9-!X5&MlI z`Ua_$Q@XfzJex>2N|qV1cS34KTMfy(2`tyw^@sFRG4f}Ngc-YJzq7lT1*B|AVUNn6Rb}7t#t7E|W34*}(b&(oh z^szR^=NNn)S&MH(G@=?W%qu>D(|+LWmcJ+IR6l0)v_&2^S=@}B7oG#{ojGaTM&#v7M0<^>Hv z?+Am`3(dJmlJ?d|Kd{&dkGN#Ay&|8hLM)q$kuzwWcliQ2S(?y-2@VC5pjnZ}f36=Q zPK?>B@yg3y)9D@ z-)Gz1A+emQmvC5~9)e9VE`Z}^P?y(9X@c0rX-#U3*I7Y`oWJ#Vlg(L*6f1jEAg#KIzRhFM;IVl(a<>N^-T_}%e;*9$n)Od8$`;EkRKB4Kn zh<_L&^`4Y*$K$mr@#8p(4!4C?4p70(&gTh1- zOl`qWI^rJr`|(t_wS_s@`rWT|oOTGf=+Gvt3JX8kHxq86dJ~g2kS)lRDH=p3-s&lsXV;gm-l-|u z7d~KO%oH1RPPNh9)?W_$o@MWyp_GE#WmO-Pi^!#`G>|^rWWbcC5tOK)I3(_~S(yLE zR!hUuO4>_eFgZ$pMME1`rdOr7kn{(Qmc6G8ES)(e;b28qc1?V6v^e1%zNYdDyF^Pq zQ-1XaJ;fiRQk8d(vdw7ej21k{7a0djTRYc2x}pqE5gg^Ub#5v>YnNX#!>SnarsG=i z9MGx`N?(r5Tsv-4ShF&Z7)3L*i4&Sys5Y-0yPflC2}SY8X-SRGm#vjA;O^n_!-6{+ z6#AIL)3XM)50t64&r|fivZIi4#6v?wrYoGcN zeOx0}Meh)HHi0`g)ZB%(ZMp|m#m-=DX58HIbhVj#HiQLz9i7K1{d7K=xvYeJFv}w> zQjL(LtWAn6Kgc9=f!c(voeq6309>h#YhC7xcqht5w?YHO8oLfl<7MO+jV8OKMNz!> z%jl_dW$v8729v`*M|I@c+dkNt6&q{Hmcolz&L>cL1!Hg*%_5)BX?t+BX&95Y3UBg> z$@65!L_TkAalX<12xgO%mdbtsN!+$fiK0wJd7W2V!vli6Oi6h7Yw|}{gO#vXDW65w zXhT9OmpQjWUjyD{%u{8xR@nt^d*jnus-dvnXfwAo`3-&Rv#vDES8*+92FAMk-PR(e zsbcc$GLOzCGA<6{zlq41HCPQFR-5PAW*PD@j(uB8Hm~{aZK%8z(X(w(Pvre3_chs@svkIBD*DQMBcl(0XYvkAmj)nbVrd&ny6oMfLb5N;jAUUKgp(AmtP(VzC3 zaQphvhmUfI9wl|z9$12`CloQxiiV;Oib8DHW$DdFF`|xehkmS2Y@QVW7n?0D6-_1> zHnVv42jy57+hO+B;|<)Woh7zA0ntBJNEFv5jxLU>V0$}Y&MigtqYFJa*- zb!>N^LSFpIH_2rQ0av|-=Mo<&Tt%fE7RI z@ImtQqVP21_w>;6j9K=u2YCidd1Pw2KSSUoe*rF(0vB6=OGClsrQph8aP={`7Tc?y z)vHm;tJ%T}8tTeVsq1v~cY<_F(E;I#heJ!s)Q9O^w<>ODT}J$dXsjSZP)1t9ZM zkVOl~awufA6tX@H**u19WBcrCc~igf-m~yI4)r-L^*JB*xj6Pg!0|=2^gbBoXb-*7 z0rG85^=*Or)-U@uBKbin{95__8npbXKz`M!el<|P(q+F2B>(bdKVmljM>t;31;CVS z;OA`KRSh2Ash%%RJni@cGAIJvkphxH0g+k(3DAJoYyofBJlM+uvRMOJqytlz1F}*B z`A0nXaDu$B!HS{w+d~c_$8M5oZnBj2dQjPK*lv;^+)l898=2i#<6TtJ>;NoR11VRj z5w|?DkVH58Y>i-fkKhnQXErS-A_^C-Qbz`@@BJ(;4mhDO%}`H)P|i@mX52k=8q0)# zncYS@%w5|hv@EonCA4ZN^es}DPgq#|N~qsTSe#|}d;aj}EMX}L!Lj6S7a!($kw5{1(f#YPi$%EEu}xk{zQ-NKD;8A+0si9|y- zRJO8*wMKN~I7+6w;vh#zg~u75B)cudNwg+uT8OCQ+CP_ZCC4RwtCL90o_M^H_>n4x zpFQalRnn%=KhrR+;& zV3?=ew@OwT%vk=E7D;6n8Sb>V;@B^fWj7M-S)TUQGd%$rNDo=fO0=@;ZOw9F&sw0$ zUQf@Sl1V3APDwk>o}o$_lu5l9$--et^Cyojp~|Qa&dAfwM8I>^ql!eM%Dq>fad0=Q zWhVn{o-4+mX^NcrR3@`dCa;ZM7a7GJM&-UFlRA%_eIS_g{48x-C*5~6n_4%YRyT(k zyMQ7iKZWZ1GwMW2yc|}J9Qur$&G0M|p#t)VH14&0_J{($_5xCl?3J?g?DAwKHn#$u zJlXIdy!JRra7KqSv< zSgXb8cT03oEJcNKKU&ACpp+@dlxu^_qEO=RQJ0~N6=|N71i@qLL$l+EhJ}AD!QK6q+F=bs5bXGbC&SYGyT3Rc&Osh(0uR;OU z{6I+&7plS4t$BB+_Q1XRe2AlAtkzxD;J#ed!{Ey9=1L5&s-CfuhfnKpE9;(mm10Mh zg4=5}QECivQWo)Qi@^1_ys9s#YqU{nkRugH&w-@$wQ3puyvKMAE#Vah5fuw-4U0)N z)zS^q?F|pt(%Cy?5rr$ryo&Bbsv$jfcq#m`cCPl3T;=yQ4Fa1gJFvcubtMi>)!kYS28*OMBTMt{vaG{2tjjWt`!sm@+c-5ae46HfJCuN%yzBfJbYMPR0Q`c)wjcnGd zY*Wx{BRa3kl&e!cZ!4^9)<6GfI3E7Zt3^OK*Ubj10j)RAY>5!I`b^W{o0;M-R}m-- zWj=3Lhc?!GwY)wreJfnU!x{2QF6fq5>*P4l;Hg((UKva|-s&pX5)#o7-Cmn84okCX z%X`|jwceJ6+I3*lg~8meRoS(#*L8y0UR3$943&NTEUD_esykDwm$SA(uCu|W6D_JU zva-`^sRK#2RnVpt#0kss=y@R9VRGKOI-c831M3`bv5f5Ppn>&eCexjF8d`MJBeyAN zecU{6+S6;_UGGYd>|?iWe^b?mkJ`-{rT<>O-wfP!zTW&CtNSmzchiq2f2!&gje^p0_9`Iv^wadT3-qkG_r9>{E(q(s?OmZR0z<(X(CLiT z6X8hA?Bbc|0)(qrCx*Cb+qp!VL9{^s4gG$>s9~WWeZI8aB2_))6IILSg->mJp8V)H zw;ie09o){SCSVvc+~`5d>Y<7nX+|EpgVsw)+bfITCNDqmqH}cVX{USSC~Z|7z5GZL z%GjgM0p_TtcIsh=AH$we;hfVdRXr6yfWoS%7OkiT8T|2l@2Q#dnUlo{)vVDx-LWLvX-MT@ zPb&<`u8&WC#?5vZd2_y(YX)_48iinl-+RW{dqL7*;xSYA4VhU&+C>AZp^B%u*|f>p zqRBU-hjiifiO}L))zpiNiMQNyv9fa+V~cE%d5+0JE`vD)yD{Xbu{zYL%B)#3g~fxc zSrpE3S%dLUn+pNil|l;Resl|B=nKJ<^8;B6r8P@!SYLQxn}{|rN#3O zZ-OxO)hs7zhDcnU~IK4cOfGExi+dK0~74q1McybdLC82s{$Y5bWbma>-un-N|fO9=ij?Ed!#ANgYr^1>O-O@AfQI5>9w&AZNrdm!=`iI ze{$oi=uCL^&Vt>h9Kr4=Y$FKPL@>2LK)*UAx{9Q@{5@)L6EccUxNUN}TBE;phkku! zbD%SOt0jA~e6lB-d;96x+RKe)BlMP2g?UqgBD@!xKYpl?!l z9X#*a8QVM{#@NL*+$0e@jE&w=65AzETtzTnd^fefda?gWakN1@d!2zrgDUQRsr38akK+7#l8267LQ_%b2!#ffulmIGX{c_BKiYK zkpp#){U?M=UiN3ETt`kA+YO4x9aF{mTZ=l;=h+Dd@>?e{hNsFkhhy}ogBJ&rQ#%@6 z3))>PR9&aE_N(vh_AUs3w2R&F&WZuCogtr-Ho|3Z3?#ct!q&yg%FPpO2lDa(Lwo~V z{R6#%L!980tw)qwWI%j)Vo0)cYG8UoOh$G}Zcezi?eBzbb*x{0a~d?MJqpNdX>IHA z?1~-m9SZ6!=<^=y?iq=LGvTvy9fR}BAw`W_+dI2^+n~dv?*a(9E_k%_h=r`NmRlnQ z3N-QzqKyPmO3-D+%s9VqJ`R7hR2JU~(fQ0GLMd+aHsXS2NpJNt|E8YKMBCTUk121w z9jkELBfrMHcNY|~I{Y}6^mT~0X}>nq2rrrj%+HOp2t za3jPZqY0y1XbwESnCP%1XmO)@U9-<(rM~POBNV+TWi_ue6t5^IoHd#R==BwTDmDl^ zGvgbp#c5s_t~}1I1>Gi>U}w%7D^%O!jaE2$FuXP&P7g~yVw9MhZrFM!8eMMLv9yr< z9VU08lD?A8;}*_+aekJikkJj$krvXsb9t7!n;{_zvO zF*wH%EaP@|JUHX})PM1r9@c7|GKdU3G+vS zX^Vk!2A1lfO7&@K5i;!j8ezuG+F@aevmVa_%oq_D!d624I!=T#ZW2^{rovA z+3do_Xz*tFY_TkD@%EO@2XH0m1i`}^8Va8g1+q066oeo06cE&;& za|T0efRwdy9A!<1Nh#A9=S~k}OGkc-01N(07*T>{Vb|*%fNiUq(9G&Z3P2!8Z-{tz z{9u?;U%t3bCg2C92MS)l(LmCj&i(OsEJ%C3^@}0}FgEIqS5tNX+VP~>T^RNFaGS_+ zH*by}>xB2*#__x}4;|}VD9J`?e}%X=E5uVsp=>hV2Yq)U5UZMX0ZHQC_tMCU^^+w` z1=^MB{bLHdZ?UPSS*W=T|B0v-f*-KdkPBAT@Pf?+-yA-I ziUp^R$#c6d#n)U1VUmVcQ>i5|yVc>>6%*&-_VmlgiNQ78$K4Hl`>P1TF$XZonhpF6 z>%>RX7f<++c4ja*i@~v@k9ptD2Z78FPA6`K@Nwo!1G!EF=R`*d17$`J6<3rBz@y$!nQM zmAWLKmu+#!2l{ko=M|Kfd(uy-OE~$JwYe}SZ-5Ihcnd`2E*&Z9nIFRVG=Z#zl1o0@ z*j<@3iM8r(N7}c5*OT+l$rsZ2W$Xe1j~bt!Znlb9odyJsYilk|3h?t>U_^QgFSE(A zcG=T?iNhFJD*d>^_?2BW^j4Gr!v;mXhQwYtp=&b#Tfttv;Kvc)Iz}0T%J|-#ej^&| zSXP{+>^aOb33z3r^|~fhh!1f;dAZ{`uR1$Mcyd9?V!t*8R#;cUW?x)|+U*Y~nmsP) zL8&`0bkJ2<2T~Znq%mR2NEw1Yrz3wclPbFZatOCKM6@pnrk5c*qaz+{W}0MZOQNtB z{Gc% z1yd=sN>}dF=+JC*zhYMsnUZBhLOU^LarQ(eSHao{onx`TXw54SSt;?8l6x;(wj>-JG`+)z0r-b5!${@I5a zSb>~eI~rKjO886IE%cB)HqFWqT-~O|p@lqOAgEkrt9ZC*H;dk()AI{1UKtriur5*& zjeLz&Sv_8~Q7BvH41I`n(vv59!7nq19w~&8@k`|CrO+%5GY3_VHtbP^WJ(dQEp@*1 zcuBF9smW0j+;r3ITilP;T^~x$053sOYOeMd85@M>sW~yb6W$Q8gCD zk2_LCi^5v@AG^dWUGiuZ19}dTyY_qp60qIiit`0p7H1qK(Y^2-xkctKHke8F zUfACG0tdj>;Y?>5@u+H<_lT|6G21keaAHM7@w77mwwHo6vHGrsy}jtdBwI3SmFbAR zFNDr4_~o;OZQs-Gy6nUJzVQuZP3x|K>hEPj_?yxt){|q|M~Qt^n`&FuegEc*R|go!U*&4)#r0vm34;d9QDg^dL~J0_<%3qDIlZ*9-{eci{$2}{gYI0y#S zus){75jIb$Mjz*p*d1EFsGK$Qv91M}(Jqv?XW{K4^7X5BNeUnA)a2Df8N#hMOyghdQxa9O=7-bJj@MP#?@X|MwV>RSp=^sNXz>!sKh)Zd%i`k$Hvb$A? zrq%aBhxDV*_xas!Cb^csa3-+;9N9`8DMFoVha3s`o%t-B-#~55mYuPsTx?R^?O4rw zQ{2jG-QJhFbB2N}N}Ww9+>Jur9hc3)$L=Pq9)YRO=B2irM@}vvMnCiS0VzgEQbhfW zuKOkS@4vWun7Inw(vH$XkwbEAG;>pw(of`bAQ-ZWGFN;!{4&P^#SO{BiNfn0zgIZ6 zmuIL)D+tt3>eUAF2tM}mx9}=v0o8!KI^Q6bB^=Z~=wuc_v3(+UK<#0kK+3?P;cpl%JISP6J|FYq~KAV*pNoak6R}+hFp`{7Gcps>v$Eh44EuuA{Nj|- zvP`%Hqo}5&uD-gmxdvKWpVrvXmJRFbZR&4t9W1Q8{mYDdH~8UpTwB8M+jW6IucQ5P z9S1=GVeHC6p?_kb2LldyczjS&aEf1AKt@PbXpVoLSAlmCq{O|jJh*xTE_C*PNl^UhZe2=H!aM?hN>Cud5vi$BN&ifOQkn)! z^I*G@pbU`P4~-3uj8FBGr*x%{!sR_nt86Q4s~ejWi#zLc^9%dC``c%GC+DZTA~&&p zrkS0`?-T6bH9!nNc%rmnWWsj;O_0ome8P}7Cn-ZRkKKRD7sq<}v;H9a$} zKr}kqx46==yt=x+(X+LUOu~3so}H2mZ^L$15MXOdV$)3acONjnl=6;aa!0$T5s;e_ z-vq{fsgU{6*{8*ah{qFEI$;hPKq?Y1x3a@~+xrE}q*7}>X=en1{%m8K$r3c0Tes3n zz!ZZo=JnUzFP@LJJ|`1LaeoLmPU#Q-n0HFkRy6x5U7&8UW3^9x$O}`$bHU`R-dK_0 zUY36r+dfTmj?q%ISH?h=Lb3hXX%GjKmXhx+IZ}npQiiz}h*&1*ZCU<8dnkQQXk=M| z1}vKAGt3AvU%SOq?A3i*tCwp-f$|M*wAN;eqs5Yeytbo-YGWmD`<|dbHUBmn>P>j} z!m@aG5E|DNtyEHeur^$vI|Zw%INF+S027K;SDx%K;KeF-Rac!IZO^sGh}GB)VIJ* z;Xh&aJN!ic_BsNBD11R(!u7So&FcmA{{6qPpjdu`oFIHmcs(Wwvc;9^3xe$F z8|W7h6bdFHagPEKg@yUYC4v)@l2THA(=(7Gqw^R&GBQKbN{X_|%X}&-i-J>Y;;Wic zALh3?71kBQlAALk+TBQEyDb{K$3`X^dgF5kTstRc0;gKWmq$0&)~9Q;x4a%&!x!n< z`6(uo4I=VW{#>FuTm&>K&QPPc1_u_Z{fOnQ<_32pdL7izZ*$EM0yHO7X&&OH0P73a z$0OUS?Lj2U)il_j7X{r2bX6kDUXr%O9DkWqV+$nd@_pdAo1v6PK^XNRjy))RkAgQq z@aSHY&evxH(Vr@~F;STHzDAI?qDR&iXN{E@0TYnC8!6cn6&8E5MQJwMfZQr=3ulm#3ZIWZkcIc`Ttdq-(0Qud^>f~tV&!jVWk8S9VZy_sOg$`I1Y;dmb^M1eS8@`ocN zKKQr-b6>nDY`Wvop1r4B2>c?!rjEfSm9`N4Nu_lmM8WQ!MzD$mjz*Y7vJh zP3BkMF5?~o3j8TA^a&14B=?B>LtY4Z@W4L5u%M{4q`c5G|59F9S5@Cs+t}P%WPhsz z*4f?FTin(R9T*rI9_b&eA06}Tnd_bJoS3YcZd;pKUfHPL8QvS8JZRWjKW)yR+}h|utmz@L&wACH)*thm_d`1C&{kKVT{f0I0V{t9wL z1Tlo&8#m$4{lUT7-rmW@*%jpOdCU3#SXPe02#Qu(GA?#R9cet_VmDZ>OgTvA{V_uCIFzf}9*uw2 z=s8wuvNqXSQL;5vYpconw!&fqweA}FdrK7_crl{2%=>@DC-Vu4e9!CnbP4^x;#175 z=jkudry%-B{ZSJhpZG4Eik0FzZ2KC8gp?lMRPH zSv5sGm=VIS>P+Q#Pi-t_k1AyncQ;Nl{DhFf$Q33(mR7_Kcs`@Syn7DXQViH@z{}sq8+*ePd;`jeXd3`zM*!aX{qBsjr zau=7DS60{7H#V<${QZN=MA3)g^v6X#)t8Sr{JF7pPE;3M4=r+)8(feVZvW=R#o;~! z1#T+f@V_)&AjQ&c^rQLdGdBJH@EPNzbHfn+*`ck8de97kXru+hTsef0@Lq@6lc^SC z5|Tlh6Gjb~HG}&Td@NIiKC4eLhwDSGYnML5jh08q{mN%#$%6GKYpnrv(WfO|BT#KD z#ZBw7y|K=i*FJPnEC;JH5b~+bc(dq(GZI}3i%Ofl#6YEWy= zn{~0Y1(|bydCup}ofFl2&ylp}{kR@_%)9#w(zUAjiQXAe3w$Rk@Z4WQ!Q*+Lj9_b0 zu(Z>OdWZmIMJ-e^ULYx)J0B-0LNnZwG*ZD#AjRhkQS4H*&EX1p3|Z**Qmo4=j#eCq zz*8&UiEcF|9&&L{JJD5CP&>(6;Z!R*L}rvSB^(!dH9lHgMkg_4Gb}wJ!=IfhF3TWY zJJWjgcQ+;R(@jM=AX{8J1>`@R0@1ZoAo|@YApZ|e;o*(IM?uh*){geT8_o7@y`BBA zZrI>x-+1O^=)}m3{qWep-1x%c>{7?V@<#jC&REaXWY6|`%j#+W-r3*{Ir0}%Wp?}Z zX!|VsGjFX?TX9fRJK{>=xc}9$O8#_X5)SycA33|axVw3P9zAjbd-}TIL;Qk3nv8Up zIZ-5|W^lY$pigpqKtehwEhXF2JHrD%EV{UuF)2T@JjcH(KdZvEKBcK45$cv#-B_E` z(%BPa^oz$tK#+n1_ZA$u693i5{Jy)xnIZP?cXxoFT9M*+gtEp8Jm`; z;359!qV&FOB_BI9JTn^UWR0%T+Vr9gk9bX>N0gR>#XzRwd&p!Ho+Q;smBQW~GF?Z9 znaiq$cRfCDuEm!`C|dZkv4&O(EqsrU5u`FGI=>c{JeergFM|sLxY0@;rf#382h-O z*u=0@w?x0>h^&-^3~)|CUVcb%iBTb>GP5i`P7T-EsmZ>jwYeQ?e=F^BCsSHi=TO@) ztp6Y4>915C9)c{w$W?wK_>;(TwYPT%yLou|xOiNm&EMA-5*qO!JTf9W#@jRA$L&}Vv=8eWL#27d>N>ypdzNmy$;+^U*+>_j{78-AAi+;O|H*X4fFyFbc(>Un|j24h!Lf)4Y%xpHs9*ANUDUVBB75}q&Q@+T*J(D?DAiM^Jlrgw{!^{y9%Dc zKXU$GIH~2n&vJ2paZ>Pd|BdAGPfqF%#k9jUC-pZu|9PBJ?REUd?|pL<9=}WDKxD}v z6;CP%bk?1^8cD^AF&v)rGna!%yx3Qfs?)%d-N|X9MXK$T@ zh#OfYX&+=P>`z`68I?2n2i zX1D5|pBHbBy1eRC~XfJAm*^}R*>mM8#9vV%D6?QjG^>j{T&15gOFL%t& zujXxLOQ`N$Ch7+yv1eZIF93v_i1U28(`1;>A5zIdbK{VmFZpRG<^8|j&-K^%zZ>gy zKe4X!chnxQ5FG0*|A=*FRFX*>6$ABlM{@TIo3Q=m$|Mo{J2Y%ebA3640X=UVYNJO> z^yenCD$EVAYOjOjCDuuDHX_V7mx_XfVnnMh`DZYn77X8NYyD6Dxv@)4{;&GJ=1*$x zbra^r?ccEeCd;AzI#+*Sw|xU1B#&cVam}F0?gR=*vo3@Xeq|3bolC5HLWw=-@=N2r z7^+K^y@_C<@es0Iq&OcE#4r*F4)VP@Fz>tL_o9fx%q#N&AKAm+Kl~)2{XF2aLYjK8 zjDdi9$VcOQ8i6VhOZ6~~=o9sDF zqlp;5Ca`!2s_-~>=_zFX&*EHgN-8|QrNiS}PJA9HE2AI=-Vg#;lo$EjPW#(E!yUf^ zQ|-S4rut7{M*nkQYF-0VhT_IGFex(fit-DK%Q9u~8L6vlYv8-O@ny=(3!v?pZ5>W=` zopZ<+q?x?jXASP?BwnF$`$JzRkC|KG4U8B>dF5$8}bmv$RUirei zLq8sXrk#-Kph}MDcZQW*5JmZ)IXzJTJ#)bogvPnyI2|jy--9eGfw1EF<2*0&u;akw z;TA+*1ZMoI_D_%;YayJ6@JgC1TnW+l{gbi;UtTf~HCBH?^wxPFA?|x>f#SFV>Om6h zmzA`NWm1T|fu&lgu$^>Ln0S0^LbzDIG+Ee(^45i5<6Y?Q5nJQW#e{hsv0w5rU57%x zKO#2QOL)ZQ`aNRv!6P>2-$m@y5C2#XfrrP2;1RnYUJlK7vrJD;AN$*fpUxz`V>z6+5Rl)4;H^fR`#~#@p!7r^+@~8qm`cn%ZE}3_`Uxtf zv6_G{;hMe;a~>&g0I?!7X_$BNyAWngx(?}YQ*G9?V!M7L-b?J>ub*3aX`2%ExV(!I z{Tya0-5n^RJ3D?VLM5Ec(m>E5sP8&bpw*ZD0VS!y1>VJhQ@PY7&a&kea0a&hNdGPc z4k8MHu!YfBwe1G6!n!%VYNIFNoHvbS{ZxJEOTz5T5=M1c3>QZ-p3JMI(xkoQ#R_;A zr&&5hItI<%N{kn_u)77`+zPWPs8zB^aH8u99XFtf591zi= zijKz*KuUJVbjOO$=-d`cF84&SpSfVk@IQ0FeFBPeCZxkwu_Ymvig(99HCJ(coGle| zAF1(*mSIh}M>d=~2Vt0n&iQa(AgMv#4PvYLF%x)@()hq>8B#w{+SKp5g9P)zjaR7(^6y)vy zfdcs_khlL&LY~m;k^;fU4ZP81%UFT`p~ThEE#+f^pm&>6du!rK@Fpi4nyzHZ=LR>X zHp~pkKn9=a4-tpSW6m-_T2`1)2|Rv$Ux3!Nmu$&Z+1TL1)5$^S&yv+YERKe!Jluz(<&Tb+ySrLM3-bR| z7yOysUB!>zvb(Fs?O#kRyn8eAw@fTe@$&CXEFAJz@#D=M%6WMFfHSfDWNGt&|8;i9 zd)2trUI>!XfH!XC%vKgcl-wv6L)C({7sIrY(-y<^idPmRjG8HzB25Rim!d2d;R0?r z6T1}SaEEF+*5!%LGQ5UNUyk=;UR{RACaRUhKnb0dq)?6YmE=gX)s>W3H>%at#9*D( zwAAGE)%48b)zysLW~#N!!a<$2tkT8wwd~5n)wP`3JJjpBjZbvf^PprI>-inbYwHEw zywn?o{SwrfXu}#A8^z-~XN`|0+^Fe_ry#lpWsA`$2Ib@N?FJ>A)f@(uyRZs_s*Oom z`l_>4Fn!rYa(hnY%^F#w;=2VAxwSXI6-L#^+>u%LAAHf<0h+KiZSqj)wAOc8FilW* zp~ODI#w}!;a>i}AIUS5px*F7%ZHyVh1sx1?HYT0Vw)Bi)^fx>6yPrIY%I|r7cFxet zJk(LxbbI~TLH`#?{euB1&8&k#IrEK!AtiU(!(p`${lgKhl&r&1y^@W?vCl0CN8_eL z`bQHMOIb&gHb)!g0}Qu_j;CF)3|MCn@e zIZ(0F#?oc82P4GzvE3ymW^*t|?PGg5D%8bwFlka#MR+`8Ut@F9?GsaVI#_^FdDhnu z!+qM?j&X5xcyo*Ax5EBk^)dNB)9P3UiR-k&{>Hx{I8j5m>W|kEJ!^!v)-<1cNv98i;?30Ciw|*^q3*bgy)+9C#1lLBRqW&93qex#HEzkF_8(sV9w}_bi zXL2TYZIF)6zhRK>?Q4VH$=X~UtkllL)Y;R~)jQCJ`mS-b>G9xT|KyC#^z6+1LSOeX zlxbva>vr?n%G%Pw>fYRO&*9NY>yO0Snam%jxeXs2Qe3hI4X&8j=LttSv8;D633Sw? zX4>A{5=P;p-eXMs;3&qN{sp^d9%@M66h(^tNJ+$mrK+=V1!DrHc25#nUm#ztI;Qsp zvhDB=iFg#-64wG&k+gT9P?(^p%-4}GIr3SH=g7gjWocRsyRH9MGJV0{GyTqwe`WeR zZvQOPueNR2*u2t#{q)o|@h+E{J_V73I6nit&weVY(^quc;Q^YzyMCVs_y{7$Sv*vls|!ZZXj}EKJk*56 zuvlvB3w(0yxorDu8U}G%uF)%j4Wqb1#o`i%yQpFNx2DYnw_c>foY{ILvDj7hYLjYiap@ zlDNHGFJbdtT+`Dd%WJ)>>+73aLp!^Ca|_3q-Mf2}+ZX4{w{K~QOuw<4n8Xq0^PQ<6 zj;89L=DDOqu;~Tcp%38s84>kls!>|Kx9_vp@p5U#&)YrdIEuZbok__&7}#L%%L2$c zA~CIH9VJ+o+|oA zy~J5wsxw<3#}Pv`VR_I$1lIJiJ3lbnnozeF6VdRwDhc_-Vt>sK*Y*l(uGOb-rr+ii zQ)|-UFLlpulhX8`>Yo3SA3hTV&3gWt9}dqVKcBzK4@(msiyu?W`?4Vl%=_^Yc&It~ z3Lb{41#olUd+sL=@6-pt`GusQuf}ES!5{3>)IvV{-%AQ(j~_`4SM_8gjgZl}mjY2B zQo70y$)m|abK&{n=!j;l3%a0IoE<@WN}T6rt5$*|H?nr3hiJK0Qh;DMWpXfb+e%#c zJ6xTFgh}c2_!Os8%Gfjo9qkPB-hj_0h{aktxrP>KbrW2n(9zl4RS)ZJ=pSf>d$69dzKOBXnd!;4 z`L5aFm4UUIjoz*0CFsu73CF29CCyct29c3?rssiRDG&n~Ym%Eb=+$5@S z4Rk7V_L|*n!?$iCU2%}I=8&gV&L}NEtV?%hn|)&VC-K8X4#A%_FlrOXj@pdq!y*2h(MOpzyEvBmCRBzY)$dIiDNJ zxyhdwL3-|%6GZ^0l^esb+D{uRR2Kp#kN8K0KpGz7zgZ!mL-~7!00!nU1|PW-JAn4@ zDgU>`QmnpmaB_gMe*7Pe{zp59&#gkNcj z?!MoyqJP`6|76?Z{&_Nb47$fRNc<0K1AZRAFxlk)+}E@gd^uX3|9*IRGX3p#>i+C5Y5-m{ zdcEEI^a&Wx?E`h^YxRGY1X`hQ`UuX{0XTktS0p|d7-11m&N4bKD#9l=Avq-_J`G61 z_z3joh62+;z@*}g^769E#Lq=FrPZnRWev5_O);g-9Z9X}^NL{w+S_jQZgdihcQ@+cS~lLqHQMNd%q%YY+I^51H+Fff09l(c_->0u;3 z9FG7*f=^7;Z`wPW@z+T$CHoIE*|O)qYO^)G8l1mQYONZ4?cHsiJ^d{j@Nb@ujEzD~ zc;OqN4!m=VoC`~fE33U-8%-Ls+dEtP&FgE2`(HNBI!;b6w!d}X^xWNE^-$pdaZ;=N zT`B$J<~X5}`j7X0oTmSGu7q;o|H(>NPQ+{jUwtuI{~5|PuKnT~O`u#O9xNLKoKzQ0 zePY-A@}uck#-2A)hSsyKiJ&({=J(l$+(a*_ekafuUmCBiMn1|Qzi6EN< z=0rYiyv~hcO$ylxVfndFg@%v+E1UL-P8{O}6SP(;&V{;@06i_4#j7)DnI*}*AD~Zw z${-k$jIE^$fkwHX%u|#qtjtmkItL2UB_>k}GCnlk&}Et@4HU&v{WG)P0i9WohY-Gh zM#8_fRrbdq`CXdK{&N9@MncLz_Uk{N^$y_iYk~L=Et>tGGyZ-S3G07q(OdW3t#Q1Fp(BuV}^rqb~=zn6avKB!)>hOfD; zt)r{Arx(&c&@eRI-8lx`0P07_<|Y?rmd0o5Rt7fOr`NWo=6CiN4_nc}j}|4r(dOGb zs9^SApYr$G^!pb1|JJGJ|I5rdv{*(!#zrht1DeQ_Ah#7A{0^EdR1DXzy5e3-1Sw@1 zuQZCV&Xw!83Fxu7fZ$8izM-LMV%)Qo@JyvmGO+<$_Ik{) zD-a7yIJT~8tA^xB6A%gynhi42yoR27s`nyEIKnALuB#t!My2{0E}fabbPxYov5d?m zV)v1O`m0#R(d6p@_ZJuOPu;^`r=Ev<5&~#G2FgI8@%P{(GJvQ6$Q=8I&|+DLVE}@d zl#wr?KEyES>0>^IzUj6t+k^a@|V~7KW16aA94Tczl{6m z%3{iS1^I==pwa^U?7!Km|C4#`g+A~x!=nh`{e3#1WzqOu<9+*QjrXnotH%p{mi}94 zowop#np|8^RF+WjStBi{wobnkTmr6YY-w(YZfkFh>`Lqd4)k{<{xjwCILPB_RSuy1 zJ&5vK_}_#0{5gm!E$^>EXjOj=5?)kXTv`@W@!2Omr?!?h`Ez6ys2SYQ*qYSX)g959 z(4ROs&<@0M6Q7uxoSB}RwPBlIo?V^xURv8)UR>E*v+;JD`ZD?Tc#`dC`(k}_|7PL( z^2gln&)dK$jrZiu{wKIB*^)72?Llr7NcVkMxi3O@Smeui?P=?x5T4)Sr9_*y{9`A7 zTqA$)l%j%H@hgFF#zK&6xwfh{>XCYdVlCQAL)c*s8YUq~ZwN{~)nPu=HD?_0 za;!AjewNkK+CbB9c6mV4$9Ah+?+iK`lq;L<(9g0YVKt8co*NGYJR2*wCHba3>Lj{G zK(77Gfan(*tbGwmfME=+i#73nA*i%FYR!Pw#pX%4YAC)6be*h0qFJLeJfZmSsNI|B zRx$pwBMyDZ+?WO zU(0CYc>ZMl^vH1eY-|J1yGR|CR(ze(4OX9T-2-_P+0X|)H`&ky@h#)O6oY@omSsO0 z&0p%D$Jh#F&HgjCC_U}}-Pro{KObAdvF*QW&JGN*2+*4IO<{)n<$y(&_bTdsx(}v| zMNZIRsAX;#+91@{jH?lpA1mBwQ2>xn1r-K4q7@bS`$4mdpTf`%ppFp-Z>avYiGPC~ z{&(Z?_wOI~u>XhN9u8*Y4_$!Oe-)@c&td)Ydzg1g0zei0<@fgO(o ziT?Cqv<)9MVrLlnySg&$&*$UvU1;1p%5`=RE34@qieG%8(*MJiw7DRRRsj6k1uKMe zqciA%LeVi9n!Ssmq?CB##4LX^-!oxg9w#>vH0B-vJipJ4l9GSRZ0qnO0h3cx zlAu?}%&dgm{2Zt_IJdYUtSG27vN8)CU&B+ER}oTP-CWYzk}t0GYdh+dl^qxv?H(GN zgeLliXZok-2Nsvcpd!|}jfJg^^}XHg@xz(@)sv;Oo^SIPU%!lA?QGxmc>f{=?6WC^ z5b;@5mMMRJ${7GIEFzETiwvgoXa-9GM!Wq18v+inL(rX7r4u%-n_y^bj zze;iZpZ?SXbJ|8+l{Bklh zxxO8R_~vr?In+Yrj{XTJTqlgAy>2by9>;TRzZ0!CZ~C7oWKh+9XE-s&^X5L=l`gNB z-*>KwVyJ|~+s&D^2EaRmFzSY#$LsXn{%QQsUJmWmMGB}Sq=6xb!)8N3DXX6op(>rD z8L$0iS|k!8Ci2w$C4hFh2EOVt%dQf`q*F~zkG!Q6_Y_psDr2zO-8r=mRKjh8-*wFuvCk8iRr{L6bZlC^>EcTR14#*B?znKJFF0yVM6n=e=?L zO0k*xwKr?Q8PoLnBiz5}# z4e47^^Qh=xX(q+{^NC&)!l4*4%)Wdx;y0_02Ch{N;Fm_3hsW0jYI(J*InT&O2@x)s zf>jyvbbzsmb6V_(?Ha+h3okaKjbb%(42e=dYG{#yYcwBZ(Y@!_b!d9kXjB7pqzsJR zTe^WJF``H&W`z7qMlr&6y-~t9pO8{LzRJ<*i?^o0>j+a^bWx(0>JIoS@IxJ01TtZC%3oau`UISLm4tP<%7hjfu%(9VoXK}{IKB*#lgk<%F666 zsCi7>5?UslQs$wh+Co23wpCPu2nd4QP`AF+%LOrU=;9{m(AB6IHEhIrJCKmz8-!sq zgq$if`=(@`SFfbz?A|bBF(z^|x@WAUR3e=|FJpda5ruW%ExEt2V9E`RV!-Z-%q=Q$ zS(an77C}!=9T&rJ*4}f^me;KsB7T>6IDAVoci1=H!@|;4yIQhc)f5IfjvmzpVpknL zEuA%h`KHaIA~%-yO?;A~C$}}u7u%%H@P}z$BR@+6Eu^v4gQVe2YUu@!q zeY3+@%f-t}mIgBuy>nJxpz-coa|MKxt%Hu7Vm7b{`_L7*x9hAdx*K)P_GXRPNIre+ z^#d}v3_)Gx+}SesRq=LiVDQ0STJNj$qV(Xym!GmtYCd}3IqQNBDlw&AXw^M!aZ!H4 z@`cqP8)^lAxcUTqJ|G3q-J>=u$9+05R3MD({;K>v=+A_in)f^FE z#8#e=2h3wO_R6A??( zJp?Je*twwBb=O(^%z|H5q%`KWIg~b&u%Gl5^<8lD{?m6L#5$XSey&Ar0~z}SI&yNH z(DNzjm_d5Zlbli=s|CuL$$CDRMx5|Dl=LKHG2v(AqXwbqnki)S0twW^foy5yC;I*` zM@0sM+&Yb(ob99GI7TSGenpN(;*53FFZ=K@);ed~RHH-s63#p`pW1;J)P&h<5u31# zh&9($f(tj{HpQGf5k>r3K-ob|_C+ink5j1$eW{Pra43OEO3$g?nN0xe+Wa_?L{@ra zM3^t7|0FRdukcZ%%g}DzE~Mq=Ozyp_X(8+Pej@nB{^)t-bzsPJ`^YQKgkUoI9sHCo z6<$Y^Qd=x!$%{enl6c&t{DM>4VrlBQviK-CGbV%J{TWtgjmi`@cd)~?E@1VzZcMl6w8{(z~dp-EM}2Q96>}i0pEur3@Qm!vp7z;R&-t@O%tAlAt}e zo-f@DTgY>#Ek&5>S<;L;QG7({HuU56hZ!?9uU5t9OqSr&hK=F%!y%J9pl6~=il}hP zqc1;rcSS-QKU)JL|8bz4b~_D{uPxCSIaF`DovDnfEwhk2(q6iqg^bo#xQ-m@-`&o& zW7bs$%N-k&-p%*R*MXBCMvlz|?-s_R>S~JQzF6zrEzXbD)zy!DdGB$zw2oO1>682F zn0B|kFJIp{H}ch`>2BpTs=j$o?!Pn0LSze+Biygl7X+Gs4e-RA%A(BN#Uhl z82LG{?S2uTUe8GeZKi7*Q>31wsUa z@UksT0~7QSRK>llZ#T1R*_9H!UZpB)H?w$tl=PfA;Rdo?4~H{g0y&J=as%NnH`I7( ze#)C0p&kVYLm=HI?2g6oO7TB55-Oj0g??Nz97HRrr=WlS%|K`}Q`F!pw~8x}XkGb~ z_o`vM{NR&+>gv(gWYFu1KQ{~BezY=(z<8{B9w>k?&pofOxKEc<#I~6l0>6(jaj^M*1Q^KxgWtf0z*nRv z*lhnHxQbc{FEA8^!M|}Itr79L97$R2f*Z4>jS9_@CS+ZM0^AR#!w8HnGIN_)uyk5bj5v9K#^6wk*?Q(HBj(|r4Ip6 zHPR%?k2svDIHoZ>Hmh6bl-TyK&ft(y{thh#!z=>;(fNV!e`h=t25-J%I^g@EgwbVR965fbMAzpesU>lP0 z#xzkSrhN1d(^RzhWKDD6AH0z0yx=i&C7q0tR#zMP)5(asHz|oGECQJL^^if~*ERivcRuWi>f@m-*!%fgeEAf*U6-6a!Lqkm9LO2^z>;^Wb|9BriUNPld zr~p2<$@AGgz=deJWeI3##Z@Jx<;oz|E%eT1Z%lyw*;O~Jqx780aXQbCs8tuwIU2t@z z_6^GdFcn%-kX6F+<^6cA9~*G@T0Y?UO`oCz*9@)y=)bx~O=w5(`|i zF(2M9sV$!0jMof|KQFd;OOO62)N`0*mLPXRrvqklp%La9`n?rm$_da>o`GJCPF+uFGT?R3|Hy7s=5HQT4oE zDOB{zcuCjs5QocFg+L=Kb1IUcs(g^or)JI!3rB6KRX{DinoP=}FB1v_hb5a*m^9+a z$lxv+XJO!md#|*Y=!}_Ln4lsvP?QDW__Ukcme62xQ%|!D@Dx$GDE`CwM6jliPl|R>)K8cPxj3sHXtfCYQ2`n!NTFQ-yX_hf5U*1iv zsMmq8Y?wO2vQk$)Auc^`8G1i-7>VvCx$U~%6FQUzONqae82omDF&-URrnu8gqM42s z7k!cbp^Mn!crP0xIW!o7lIU15Ut3tjs(`7e44e|GjA~$xu*Fzu`H{tHkJ1enyq6kw z*|4AB&(gR(Dt1>!5BJW^swg>_;r8Ri8}3l1-nYpI+i|q<%m=-eCC9aMjI2LUYD`~C zd?}jEWRhxQwGJ;u{?OHOvg8on#L?t}W#byYpBl!x(g>$@wR+dev{mzT6jWc_!*Jx* z8k4o?*2=NCZ$x8uhm_6@ALM2iR@t~>*SLoJ{s^4A~`?lR`pY*!0Or&bxt?B+V} zOV1zgwK`1nS|SMOu*xer*=u^x3lO8n%u0g<=*S_au8j=2&mH?G0c4?;&(<8yg4|5D zqLq{R2+QrnvyU_abNN5(s8zXYAgmJl5H(YwqKgfNBxEzI>M0RjPjrPR2JB!E!$x zGA@Zr))_Zrk8XIUaR^Q+d+|09=@YsxS33fjt&q1ibj0owoYvAzPu+T*%-NXtF>&cuJIwgr?tok(LHW)H4-Wut) z{Yuf-k9LB%q}-r7ccXj~RPnTYB5N~sPhx|qn|f0vB<9GHLX>6F0X0F@H@}d{sh(ea z7N?+3EWe64^|OQvju=ik;U>Sv1eg8`Cmz|uecKoOF%{Fn7i`Sr-%^xc{hTV+cvxAg z{0ZSj8NkO!G9wdnE>RqKsT%&8EIC+eU{(q29RI#xoALRLx^!=C1rO&Aj^e16H%evdnAsl!(uL&Qr)badO4+?)MHVE+)dLd4Oh$_BNj}X670%dW;C_)6 zaIV|QRL>y307(4Ul~=EwtyOR)mEMukDOq%hh6LJG>!V2_s$Jr}SRq&2ky2+PtS`c} zQ+LeZmN=440EzLiSdVG06zpO*t*hy&HVCSv7ZSJtOpD#fnOA&y_nVWh%OB*H1a+?& zCX4z56X|CwkzV%uc2Qwgg-E_K#){|fMi6xfe(kx5SH0fztQCgFQ^bVQ_H!?9uS1j- z8`ARtn7-FY;HcnN+Z1>Wy-V-Faq&;!unpq}-nE`ZeYD-CnwlRJU_gj`pbDbJTEOOP z1w?4Un$QGi_bJI&r^*>3@GxpUv6DZKF}lZiJ%2OmR##2&J!VgMs&~vnV2V;U1|UXy zJB~_~n-ePKE6&$AKJV$2Loa0YR^{nL48w~;@)1k4<@G5INC$yLMgX0s_OyoKcMSaU zeHjz(iQv}nh1^0%+N1*G5}abJaFJFtnBS&z2zfvXa<;|~?lW1JFTfujPHiMi1jbvh zqcxJSEfKZ_U{TsH3w4C7q(2HyRu$Hi*<}DU<{GEoB3xpCIIW)opH3Ic^Od&fy%!7j zn5g6BYxq!Zr+bu2V4>|-NA6{7MEYvJht~xwZsgRAZ)qWLl((h;gUEVIdus2Jx4zBJ z&!cZ~a(niwZle6Fl0nXNdsKbVUgU{}hvyp2+EvbvmLu+qR5(N|-cCIIV-wc9y)17? z8KbqeIoc#@YJ!7v&}&freNfxbBLFbTF_e zhR>Ymb22YV3|5Zp`9$ZozB)@Zl=Ng|M zW{9*82VCEC5CLjueeNVaI?{N)Q}pDWvpcWz77q3}uD4F;b4NM#_@3+=V(&yI<*LHx zO0n-u6(UClwDD8)lZbI@AoQcWcA@TfvrV#*=EG(RaqdcR3*LKh)x>sjCi1T+@)g-f z`+SOaDd{PpX!kSOr^Lb&1qStvRKQiA4HAm?mL&d_tKD^>b)%&38$KV=>p)wjfEZ+t z`R;(;;Md|IKJ~o;48%{BfObC^eTYzkoKSETEb-o;cq%}GT{Rz@0t1MDbp+N7333Yw z0fvM@I|9%D>Ih668VBt!p&6Rw777H0rb0r~=R-5EL$is)a{0sZHNy(s!a%^V5=dCt zd>FK)LnU!InE!vJDKK&5B!A?z=D*VvI3nc1l}XcYP7&G^*zeO7#p{@;NX95J{s5DC z0FdybEHRn_1RyCDofZs;c^)7I@~S4Z9EFb#qzZb{AN}*ndc!pukviJC|I^R87&Q<+ zT0dZ;*HWzS142rq2U8F)k#`7hV46j=#XQz6fAl@F*ALCud#Y&p&S(WbuZH@-W(!}$ zn}{!(er)~D9P{z)DE`90sQDLuB320w$o_7USmc)eA@%+*-TW1qox}MOI#A+pm}8dr zJXuo$S_(gq&RKuw1?b#(>Rb8R5kHXt#mw`0z0$C+tB-97eoNvQiPQAaxgGNBBx9$U4}7X@Fsn1!DR(Cft@(x zCBKWAMuu;4rfqVe&7rSriSt@35jS#(VGB&u}^&FE~<>5kw9#=(|#)z%WVRzc>{619R&)s}=cJcPA! zcgTaz_*ZJ$;)$5%0jl4p4IZXdenMR@xzK__6V;Y92SL`22i zL|rsNF}hmiWJllO<4ZpP#G^wOKheO_L)A8qT_*6&`%*)Xc&U;mrO$YV&8!A!V%h=u zf}|Jqm-#CGO#&J_jX?m7R40u|Qt4?@>5pnHJoYWN1sa|OGTu8HJ_PSJj$}69X~m0c zjReR6b~Jr>-o?zc1|}kZGNhd$eKPALw@{;%S=&~+EfWLN6hhr}&7*ybsArAdTxzD5 zXj4ye$x)BHPRs;2?{ zn0*dWxvP_xcKJa~nT~A_EP>C~youY`t!R+PDqbLHn=^ zTV_c6a<*G0r!``ETKx)Ic{|a@yIa>;+cupLE_M2xoZH%C+A?x+q+HKh!Kx< zBg5P4m*k(Ns|A(z?9;(sUe9uVss=JKW2k zi_$Yp+dA`5pJW4>JMB$M=-(ce-euEhH|)zz7(>%*Gd z-qD%r*5ihT*83*?=`86EDQ8k(3-pY0o1a$8v6C>fkt-CZ7@-`m;i@j=_%Tsq!8 zyS!>H_*{FRd-o$u74|!#J$8rkwfV)ZMvP)-@VxeOTi`y_@+t#K{@9`l_pN%R03qFV z)i#^s%A!_Yk)w8@GqW0a`EqZGFFty=$Mz*&k%9t~PQ@~F!c!#^o{2=+{aU-yptqOUVwoV+XvXyGGgbf<Y z2ODoE)}Fc-cgY;BhG)GJ>|ehe*0$NT^p-o=+P$lA!J#a^<2msF>PDtQDSqc8$tYcD zcOCc3uj@rVSkZcdH%D7z8VbU;9_|;LBOiS}{P^^0tu1+(WMTbRjND*KN@#K{1}!kKhmWQlkxU3AQO#b9!=c><3HxBl!H=MCM<9w>Tm}08 zb)bg!zNv^ZizI!Lq;{xjl744F9-D!{wHdIzjJhb@vH$sReRGD(SE&Du%{1zMw(q6% zeuf`Kszq)X2TLCZfNPN{3!rZgo5QaM6HA%ySQ1Q;@yq|l9nBi*e{b_ zXkC^!Dq~abKAhW*3-!OT`7A+P6#!T7VqHqvbQ>m8*+%0Z0Xn0*JHqV{@Q4QW5r&u6 zm6#8e*H`$FmO^U0OG-ccu$>{p9iKOr;h@Mu&2kDTSj(G+!K5Fm$KNiMwjLubaeSUN z_h&C#rby>3*-i?t>_X=wdtU)wKB{|oJhsz?F1q$kHL*)mfPhIZMWj7@QMk_zk+EPV_cGQIjoYIcqW4R7rtlm`RRql)YY8@;8@&&pvyD!NT zLX>KxqSsF7?t^criv*PFAsI&+?z9VsHGM)}1aQs`;ThMK7ENah*dBpW?b`uTZQ(bg zdV>P1ZOwCBH%d|7=if9@wTOjOG2VQfdMAEiJkRxY0^6&ydIPRXviF2tEQF1grrDFJ z^&1ICXr~`_`?!=K9{*WrT@!6D5V4whkVX}7pvqZi2_eb-U}zYThl#@}I8YTE89`<| zG=43_=%S$Y(bSaO0(szON9iFtAU>H=5;YcE9mBuyo{FW~9FEPf(nZDJtkRHqXwkvW zuPWbgkrybcZ48Sp0m@+Gfed0t#U%7MBU9s8NnJF^JA{$ZV7USZWXzQf6KwP7%|QLL zwGJ*=WCpxKsDu4GN?`-C7Q9Kt0+AD^;Q;h4>@LP`+eMYQVZUvDMpx+%5^=Fu)7!#l znxh*T@rH-8`<(L)e9xY8WKPJMz>!Hydu@JIk2N(6PezXxY#2jo(O2FnMgs*=&L znX@RQbl{9CCwC0*um``C#c&)?cnwp?=}@9<9Cw;8!ZXkubH=4V@X;$=Q0oL}IqIWb z?e^pCCT`?YIohWu@zf7OOy_=sYIjd~w36Ovtr|*|cb1PuCWNyLBp{Z(*B{lgXS2MQ zcWJ0t^jP0CCe-z^MkUxI-n zW#)WC3((AFbv8!1rR2NC$<)glkX*U7#?azy<7I7SWVx-yyQRg&%Q{H;muANyo%Z9y z(vGy`&rup0MYW$HEs1s=PbH_TZZF79Gpx0c!_hV{=Q5gQ-rG*8ha`G`DeQf_!`w-# z^~J@d!NZKwU@P(2TTG|Cm4p({P*PM2`r35v;fhZUsT#tCSFOPB7Nk|5#_W(dD!)0~ z7!2+>x6-==Y$Fe9%5G+dBII%K(vN~JRIf7ewKJKWR~UDkYeGy zuVlujME|WJb*_8=YtrJ}JKp5tjd1Ryn}t>Zm7pRv2}1>ogF#*G+XBAK)OQ+asO1Z| zz}1x+IP%C;!6v+6_o)2@Ji7^|AZ0Jf0Q*w}HVenbh|&Dr$l|VhMP78^Fr`E(&Zna3 z%MAg%(=;1VqX&yN-6QS6;a9cXDnE{do66?82){Yp%|p4?s&q3uUQ3I@7a2Y9K8Oj3 z`Lr#@X}m7U9rHW+U4;wd#p77j{bYM!O+Cx{__>}|j#vDNd{3ZFIzkD8>fa6XZ#xn= zO{x$OFb4Gy8FMWfc`r0a1aiib+`D2@(v`nnuy;1D`h@!tq5oLVp?Lsj>)dg9UkvFD z{d?V*O-+86Hyj0y$T3#J4>vI3yiWN9URDSCfy_-qC{A~;Y$WSbgdnB4_`C{hci#HT z+WL*3F`aj`71Z{QH=N%mZ?CGi3G^vNxhS(^9Wm?bv@8l=;WeUF+ha8?X1Jc0RqMJp zo|3Jsc%M8}86^3LKD->q&uZ&Fd3G3S@oGCb2v$!~|2&G(b>ftxv;*Dy#Bi-~?c)7O z`_hjS?hgcea}}5MKCLG`jzULz!0@^F%v0Ak+f~Um_p!N*tEwxZ3fX5a8)HAOa9=mW za4U%HqOaV%4Hh1MKIWFK_0u9gx_J!Tc01nHdS%<@wP=-fG8OJ~#b3{M+lTe@q#pbF zj=}qIZ0%;bwC(4YwWi~_XpiaS*2`UFpZjDjT}6QIDYCU801X0oG7mrmA))RAaQS_8 zVSMr3d$$z?D;r{mgZ zsBPc$Gs&gmoTn$Fp;mV!TeqjN+VrQ$(PGyO5I1I^=_KPIrn;J706S3I;%KvR(LVPJ z{5C<$(;2vZM!WHxLZd2B8zpE$Nn4uV|Ctm6<)nYjDc)O9kfM~fm79NuC^~Tm3PKzy z#u=G|WpGfXwsU{5Y+P_$Ww0I4pXEF_fH>zgdR*u6ss5!LS2g(CJ#a>BgV<^#}ai8qkv_oF9@rY3U4_J3(eL918JC%LJvhr zL_4**q{yRY(Xy$+Mj5c9AQ3Tt2d$-H4WUMzZ~r6MHbB(OO!Ut-YB{;tW`cquDS!GY z_!SftSJl-+8XB{zK~0Y;0!T|O)IwxnFtTT8bZoo}3WvWRJrK%|t zsPwKJPOm{k2sjkC;`B*8Jh-@z?v4vv8%2b%A&gkc*vuSXH>UbNZ-8B@N<@^T>?!hG z;R(E9HY;htAJkYxVnxCEPnalKKv0i>f@Bg-BXwy$6Pg|ml0wdsT>32vI7&8mvRhWt zu{5X!QUS`eaZ7;|swpQkZH^cb0Y6JTt)z}dt`B%52Da8#i%1y|!7r*5gr8>q5SvLH zI~on`*LQX}VKS$p-{kSI`h%XBbFG!U8nGC=-a(Z}_zM5k;q@w7MiANqMj43%ZAZGG zM^M2_r&X`DO4ZN-_aLH=>?S_MT9ek(+5VVdyE1Vm>IqjlcuGB#?;9I#dWpjN5Ylc% zHR1VINh;Pn`*Jg5rJDLC*Arq+PiWIF4>uwI{;X$X6!#_PM@uI&m%6GW5tdVdVjN@* zMiW17_9Ow_qTqA@{Lc;K7Z;(p-FO_s1#8}2IGWTSd#QoRfvL|OvjLsVQ|nA3mNRHR z#D2idAlBPHYWon+edBP%U9MbqcYGxTyv+W34Nu`HVlt@z7)c@U*&(8N5FGB-SU~+-2l^yU8O1KF1Sjd;6f>8k zqKNh{H}rlvEpSSBQuvE+SznvN?&UhL0DS3O+;4KTELUIBW?Q_kqv5q7^g}?+4njLb z&!;&HB@yR*w0%5R>dNy&c)1;4?X6)GSO4^esX}RRu{63GJUNPX zwJsgsbO@k|wdS`fbT=Mi1T6k6&NWP*+$^r@O>IOa8sd1`Lq+roy}V>n*&_!Czqw#n z5a;sFss-p*DkBI-CCXB@-AV;uKx-yp#0=Vw{o3SE!TKTV3f#~B&~iU{wAV`Z;r2sO zILbIe2VJw_Ar_D20SZzWM;LN+y~s^|dE4Zx>Jh%Q`uqx>`sC91HB@viTH4^-PD=Kz zDkNHBNVpk}0X_GJI^q)>hgubbTMfv=3$b%_6sBXAjF9~^`y^97RbqEF4NvFUymCDU zfNu4t{-PW|*pr%|>>*IMp&Xdq4^PE&(PHiWk19xX3>i|XR{jsk&TyG_`}L-5i%5!& zSq#)wI!|9DO5BhU$!6lW9s5JT`cbUJ0DKHd(8tzyu96tg;Ls4M-*2LlS9YCXKY z@I$V!52QWU>6T&Yl$<^d*p{Krak*}z6sQs%)K(=#ap;cY@eK%7q{LMMqQqhh2g}zO zdZ9o-H0{>dg4-MIvJxeJ1T2wxoyNp_kdWKy?1(4$0jTx-9rG#nK~AqG49U*PW;&#O zf&jFXLMXieoYG+5;fOY4y85n2xHsj1ki>O5-vVmm2ahmW8e(*6FbpQHNnn$v^MvW<2=KgR?@Q-w0e%e;)6LBRwV_0PHk5Yd1!wOHM5W}aPMZY#T(;L}-d z8-zAvG#XFyd+OG@g=gmSc<_Y2kR;LaeK|ii+JkkZ3WXg7x`kBBmA-KhEK(i+n}9QJlA47F_04@UjRS#fp(7szow0CE6Dynt`&MYMn=^d&nz>6|V zxl--9!P&aTi}Dl^tI<;u6<`TmAq#ml&*1HB>+Qv7fSe6UkbAc1Hoioely&ev-h8jX zWt9~MoDn;{FUtqv{uKY#q_b QL3NUUYU=-w$V>$P2TZ7nQUCw| literal 0 HcmV?d00001 diff --git a/docs/ja/index.md b/docs/ja/index.md deleted file mode 100644 index 6dea5f6570b..00000000000 --- a/docs/ja/index.md +++ /dev/null @@ -1,142 +0,0 @@ -# ClickHouseとは? - -ClickHouseは、クエリのオンライン分析処理(OLAP)用の列指向のデータベース管理システム(DBMS)です。 - -「通常の」行指向のDBMSでは、データは次の順序で保存されます。 - -| Row | WatchID | JavaEnable | Title | GoodEvent | EventTime | -| ------ | ------------------- | ---------- | ------------------ | --------- | ------------------- | -| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 | -| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 | -| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 | -| #N | ... | ... | ... | ... | ... | - -つまり、行に関連するすべての値は物理的に隣り合わせに格納されます。 - -行指向のDBMSの例:MySQL, Postgres および MS SQL Server -{: .grey } - -列指向のDBMSでは、データは次のように保存されます: - -| Row: | #0 | #1 | #2 | #N | -| ----------- | ------------------- | ------------------- | ------------------- | ------------------- | -| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... | -| JavaEnable: | 1 | 0 | 1 | ... | -| Title: | Investor Relations | Contact us | Mission | ... | -| GoodEvent: | 1 | 1 | 1 | ... | -| EventTime: | 2016-05-18 05:19:20 | 2016-05-18 08:10:20 | 2016-05-18 07:38:00 | ... | - -これらの例は、データが配置される順序のみを示しています。 -異なる列の値は別々に保存され、同じ列のデータは一緒に保存されます。 - -列指向DBMSの例:Vertica, Paraccel (Actian Matrix and Amazon Redshift), Sybase IQ, Exasol, Infobright, InfiniDB, MonetDB (VectorWise and Actian Vector), LucidDB, SAP HANA, Google Dremel, Google PowerDrill, Druid および kdb+ -{: .grey } - -異なったデータ格納の順序は、異なったシナリオにより適します。 -データアクセスシナリオとは、クエリの実行内容、頻度、割合を指します。クエリで読み取られるの各種データの量(行、列、バイト)。データの読み取りと更新の関係。作業データのサイズとローカルでの使用方法。トランザクションが使用されるかどうか、およびそれらがどの程度分離されているか。データ複製と論理的整合性の要件。クエリの種類ごとの遅延とスループットの要件など。 - -システムの負荷が高いほど、使用シナリオの要件に一致するようにセットアップされたシステムをカスタマイズすることがより重要になり、このカスタマイズはより細かくなります。大きく異なるシナリオに等しく適したシステムはありません。システムがさまざまなシナリオに適応可能である場合、高負荷下では、システムはすべてのシナリオを同等に不十分に処理するか、1つまたはいくつかの可能なシナリオでうまく機能します。 - -## OLAPシナリオの主要なプロパティ - -- リクエストの大部分は読み取りアクセス用である。 -- データは、単一行ではなく、かなり大きなバッチ(> 1000行)で更新されます。または、まったく更新されない。 -- データはDBに追加されるが、変更されない。 -- 読み取りの場合、非常に多くの行がDBから抽出されるが、一部の列のみ。 -- テーブルは「幅が広く」、多数の列が含まれる。 -- クエリは比較的まれ(通常、サーバーあたり毎秒数百あるいはそれ以下の数のクエリ)。 -- 単純なクエリでは、約50ミリ秒の遅延が容認される。 -- 列の値はかなり小さく、数値や短い文字列(たとえば、URLごとに60バイト)。 -- 単一のクエリを処理する場合、高いスループットが必要(サーバーあたり毎秒最大数十億行)。 -- トランザクションは必要ない。 -- データの一貫性の要件が低い。 -- クエリごとに1つの大きなテーブルがある。 1つを除くすべてのテーブルは小さい。 -- クエリ結果は、ソースデータよりも大幅に小さくなる。つまり、データはフィルター処理または集計されるため、結果は単一サーバーのRAMに収まる。 - -OLAPシナリオは、他の一般的なシナリオ(OLTPやKey-Valueアクセスなど)とは非常に異なることが容易にわかります。 したがって、まともなパフォーマンスを得るには、OLTPまたはKey-Value DBを使用して分析クエリを処理しようとするのは無意味です。 たとえば、分析にMongoDBまたはRedisを使用しようとすると、OLAPデータベースに比べてパフォーマンスが非常に低下します。 - -## OLAPシナリオで列指向データベースがよりよく機能する理由 - -列指向データベースは、OLAPシナリオにより適しています。ほとんどのクエリの処理が少なくとも100倍高速です。 理由を以下に詳しく説明しますが、その根拠は視覚的に簡単に説明できます: - -**行指向DBMS** - -![Row-oriented](images/row_oriented.gif#) - -**列指向DBMS** - -![Column-oriented](images/column_oriented.gif#) - -違いがわかりましたか? - -### Input/output - -1. 分析クエリでは、少数のテーブル列のみを読み取る必要があります。列指向のデータベースでは、必要なデータのみを読み取ることができます。たとえば、100のうち5つの列が必要な場合、I/Oが20倍削減されることが期待できます。 -2. データはパケットで読み取られるため、圧縮が容易です。列のデータも圧縮が簡単です。これにより、I/Oボリュームがさらに削減されます。 -3. I/Oの削減により、より多くのデータがシステムキャッシュに収まります。 - -たとえば、「各広告プラットフォームのレコード数をカウントする」クエリでは、1つの「広告プラットフォームID」列を読み取る必要がありますが、これは非圧縮では1バイトの領域を要します。トラフィックのほとんどが広告プラットフォームからのものではない場合、この列は少なくとも10倍の圧縮が期待できます。高速な圧縮アルゴリズムを使用すれば、1秒あたり少なくとも非圧縮データに換算して数ギガバイトの速度でデータを展開できます。つまり、このクエリは、単一のサーバーで1秒あたり約数十億行の速度で処理できます。この速度はまさに実際に達成されます。 - -
Example -``` -$ clickhouse-client -ClickHouse client version 0.0.52053. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.52053. - -:) SELECT CounterID, count() FROM hits GROUP BY CounterID ORDER BY count() DESC LIMIT 20 - -SELECT -CounterID, -count() -FROM hits -GROUP BY CounterID -ORDER BY count() DESC -LIMIT 20 - -┌─CounterID─┬──count()─┐ -│ 114208 │ 56057344 │ -│ 115080 │ 51619590 │ -│ 3228 │ 44658301 │ -│ 38230 │ 42045932 │ -│ 145263 │ 42042158 │ -│ 91244 │ 38297270 │ -│ 154139 │ 26647572 │ -│ 150748 │ 24112755 │ -│ 242232 │ 21302571 │ -│ 338158 │ 13507087 │ -│ 62180 │ 12229491 │ -│ 82264 │ 12187441 │ -│ 232261 │ 12148031 │ -│ 146272 │ 11438516 │ -│ 168777 │ 11403636 │ -│ 4120072 │ 11227824 │ -│ 10938808 │ 10519739 │ -│ 74088 │ 9047015 │ -│ 115079 │ 8837972 │ -│ 337234 │ 8205961 │ -└───────────┴──────────┘ - -20 rows in set. Elapsed: 0.153 sec. Processed 1.00 billion rows, 4.00 GB (6.53 billion rows/s., 26.10 GB/s.) - -:) -``` - -
- -### CPU - -クエリを実行するには大量の行を処理する必要があるため、個別の行ではなくベクター全体のすべての操作をディスパッチするか、ディスパッチコストがほとんどないようにクエリエンジンを実装すると効率的です。 適切なディスクサブシステムでこれを行わないと、クエリインタープリターが必然的にCPUを失速させます。 -データを列に格納し、可能な場合は列ごとに処理することは理にかなっています。 - -これを行うには2つの方法があります: - -1. ベクトルエンジン。 すべての操作は、個別の値ではなく、ベクトルに対して記述されます。 これは、オペレーションを頻繁に呼び出す必要がなく、ディスパッチコストが無視できることを意味します。 操作コードには、最適化された内部サイクルが含まれています。 - -2. コード生成。 クエリ用に生成されたコードには、すべての間接的な呼び出しが含まれています。 - -これは、単純なクエリを実行する場合には意味がないため、「通常の」データベースでは実行されません。 ただし、例外があります。 たとえば、MemSQLはコード生成を使用して、SQLクエリを処理する際の遅延を減らします。 (比較のために、分析DBMSではレイテンシではなくスループットの最適化が必要です。) - -CPU効率のために、クエリ言語は宣言型(SQLまたはMDX)、または少なくともベクトル(J、K)でなければなりません。 クエリには、最適化を可能にする暗黙的なループのみを含める必要があります。 - -[Original article](https://clickhouse.yandex/docs/en/) diff --git a/docs/ja/index.md b/docs/ja/index.md new file mode 120000 index 00000000000..facf2d71348 --- /dev/null +++ b/docs/ja/index.md @@ -0,0 +1 @@ +../en/index.md \ No newline at end of file diff --git a/docs/ja/interfaces/cli.md b/docs/ja/interfaces/cli.md new file mode 120000 index 00000000000..04588066828 --- /dev/null +++ b/docs/ja/interfaces/cli.md @@ -0,0 +1 @@ +../../en/interfaces/cli.md \ No newline at end of file diff --git a/docs/ja/interfaces/cpp.md b/docs/ja/interfaces/cpp.md new file mode 120000 index 00000000000..581e50e774d --- /dev/null +++ b/docs/ja/interfaces/cpp.md @@ -0,0 +1 @@ +../../en/interfaces/cpp.md \ No newline at end of file diff --git a/docs/ja/interfaces/formats.md b/docs/ja/interfaces/formats.md new file mode 120000 index 00000000000..41a65ebe579 --- /dev/null +++ b/docs/ja/interfaces/formats.md @@ -0,0 +1 @@ +../../en/interfaces/formats.md \ No newline at end of file diff --git a/docs/ja/interfaces/http.md b/docs/ja/interfaces/http.md new file mode 120000 index 00000000000..fb293841d8b --- /dev/null +++ b/docs/ja/interfaces/http.md @@ -0,0 +1 @@ +../../en/interfaces/http.md \ No newline at end of file diff --git a/docs/ja/interfaces/index.md b/docs/ja/interfaces/index.md new file mode 120000 index 00000000000..61537763cac --- /dev/null +++ b/docs/ja/interfaces/index.md @@ -0,0 +1 @@ +../../en/interfaces/index.md \ No newline at end of file diff --git a/docs/ja/interfaces/jdbc.md b/docs/ja/interfaces/jdbc.md new file mode 120000 index 00000000000..27dfe0cfa5a --- /dev/null +++ b/docs/ja/interfaces/jdbc.md @@ -0,0 +1 @@ +../../en/interfaces/jdbc.md \ No newline at end of file diff --git a/docs/ja/interfaces/odbc.md b/docs/ja/interfaces/odbc.md new file mode 120000 index 00000000000..5ff7610e061 --- /dev/null +++ b/docs/ja/interfaces/odbc.md @@ -0,0 +1 @@ +../../en/interfaces/odbc.md \ No newline at end of file diff --git a/docs/ja/interfaces/tcp.md b/docs/ja/interfaces/tcp.md new file mode 120000 index 00000000000..a0529a856e4 --- /dev/null +++ b/docs/ja/interfaces/tcp.md @@ -0,0 +1 @@ +../../en/interfaces/tcp.md \ No newline at end of file diff --git a/docs/ja/interfaces/third-party/client_libraries.md b/docs/ja/interfaces/third-party/client_libraries.md new file mode 120000 index 00000000000..5320bbe1e16 --- /dev/null +++ b/docs/ja/interfaces/third-party/client_libraries.md @@ -0,0 +1 @@ +../../../en/interfaces/third-party/client_libraries.md \ No newline at end of file diff --git a/docs/ja/interfaces/third-party/gui.md b/docs/ja/interfaces/third-party/gui.md new file mode 120000 index 00000000000..ef7bc904197 --- /dev/null +++ b/docs/ja/interfaces/third-party/gui.md @@ -0,0 +1 @@ +../../../en/interfaces/third-party/gui.md \ No newline at end of file diff --git a/docs/ja/interfaces/third-party/integrations.md b/docs/ja/interfaces/third-party/integrations.md new file mode 120000 index 00000000000..9cd0a21e676 --- /dev/null +++ b/docs/ja/interfaces/third-party/integrations.md @@ -0,0 +1 @@ +../../../en/interfaces/third-party/integrations.md \ No newline at end of file diff --git a/docs/ja/interfaces/third-party/proxy.md b/docs/ja/interfaces/third-party/proxy.md new file mode 120000 index 00000000000..877f1b51dab --- /dev/null +++ b/docs/ja/interfaces/third-party/proxy.md @@ -0,0 +1 @@ +../../../en/interfaces/third-party/proxy.md \ No newline at end of file diff --git a/docs/ja/introduction/distinctive_features.md b/docs/ja/introduction/distinctive_features.md new file mode 120000 index 00000000000..9cf00a2a00f --- /dev/null +++ b/docs/ja/introduction/distinctive_features.md @@ -0,0 +1 @@ +../../en/introduction/distinctive_features.md \ No newline at end of file diff --git a/docs/ja/introduction/features_considered_disadvantages.md b/docs/ja/introduction/features_considered_disadvantages.md new file mode 120000 index 00000000000..45d3cdf563a --- /dev/null +++ b/docs/ja/introduction/features_considered_disadvantages.md @@ -0,0 +1 @@ +../../en/introduction/features_considered_disadvantages.md \ No newline at end of file diff --git a/docs/ja/introduction/history.md b/docs/ja/introduction/history.md new file mode 120000 index 00000000000..7004e990a59 --- /dev/null +++ b/docs/ja/introduction/history.md @@ -0,0 +1 @@ +../../en/introduction/history.md \ No newline at end of file diff --git a/docs/ja/introduction/performance.md b/docs/ja/introduction/performance.md new file mode 120000 index 00000000000..cb2912bcb81 --- /dev/null +++ b/docs/ja/introduction/performance.md @@ -0,0 +1 @@ +../../en/introduction/performance.md \ No newline at end of file diff --git a/docs/ja/operations/access_rights.md b/docs/ja/operations/access_rights.md new file mode 120000 index 00000000000..73463029569 --- /dev/null +++ b/docs/ja/operations/access_rights.md @@ -0,0 +1 @@ +../../en/operations/access_rights.md \ No newline at end of file diff --git a/docs/ja/operations/backup.md b/docs/ja/operations/backup.md new file mode 120000 index 00000000000..1003fb30e61 --- /dev/null +++ b/docs/ja/operations/backup.md @@ -0,0 +1 @@ +../../en/operations/backup.md \ No newline at end of file diff --git a/docs/ja/operations/configuration_files.md b/docs/ja/operations/configuration_files.md new file mode 120000 index 00000000000..a2d73dbaa25 --- /dev/null +++ b/docs/ja/operations/configuration_files.md @@ -0,0 +1 @@ +../../en/operations/configuration_files.md \ No newline at end of file diff --git a/docs/ja/operations/index.md b/docs/ja/operations/index.md new file mode 120000 index 00000000000..ce854687b86 --- /dev/null +++ b/docs/ja/operations/index.md @@ -0,0 +1 @@ +../../en/operations/index.md \ No newline at end of file diff --git a/docs/ja/operations/monitoring.md b/docs/ja/operations/monitoring.md new file mode 120000 index 00000000000..515ae8b4fff --- /dev/null +++ b/docs/ja/operations/monitoring.md @@ -0,0 +1 @@ +../../en/operations/monitoring.md \ No newline at end of file diff --git a/docs/ja/operations/quotas.md b/docs/ja/operations/quotas.md new file mode 120000 index 00000000000..1c52cdf1e91 --- /dev/null +++ b/docs/ja/operations/quotas.md @@ -0,0 +1 @@ +../../en/operations/quotas.md \ No newline at end of file diff --git a/docs/ja/operations/requirements.md b/docs/ja/operations/requirements.md new file mode 120000 index 00000000000..a71283af25c --- /dev/null +++ b/docs/ja/operations/requirements.md @@ -0,0 +1 @@ +../../en/operations/requirements.md \ No newline at end of file diff --git a/docs/ja/operations/server_settings/index.md b/docs/ja/operations/server_settings/index.md new file mode 120000 index 00000000000..1d1a0585a42 --- /dev/null +++ b/docs/ja/operations/server_settings/index.md @@ -0,0 +1 @@ +../../../en/operations/server_settings/index.md \ No newline at end of file diff --git a/docs/ja/operations/server_settings/settings.md b/docs/ja/operations/server_settings/settings.md new file mode 120000 index 00000000000..19cd2e82ce7 --- /dev/null +++ b/docs/ja/operations/server_settings/settings.md @@ -0,0 +1 @@ +../../../en/operations/server_settings/settings.md \ No newline at end of file diff --git a/docs/ja/operations/settings/constraints_on_settings.md b/docs/ja/operations/settings/constraints_on_settings.md new file mode 120000 index 00000000000..4dacf908662 --- /dev/null +++ b/docs/ja/operations/settings/constraints_on_settings.md @@ -0,0 +1 @@ +../../../en/operations/settings/constraints_on_settings.md \ No newline at end of file diff --git a/docs/ja/operations/settings/index.md b/docs/ja/operations/settings/index.md new file mode 120000 index 00000000000..fc3968d1f1e --- /dev/null +++ b/docs/ja/operations/settings/index.md @@ -0,0 +1 @@ +../../../en/operations/settings/index.md \ No newline at end of file diff --git a/docs/ja/operations/settings/permissions_for_queries.md b/docs/ja/operations/settings/permissions_for_queries.md new file mode 120000 index 00000000000..ce8473bf01c --- /dev/null +++ b/docs/ja/operations/settings/permissions_for_queries.md @@ -0,0 +1 @@ +../../../en/operations/settings/permissions_for_queries.md \ No newline at end of file diff --git a/docs/ja/operations/settings/query_complexity.md b/docs/ja/operations/settings/query_complexity.md new file mode 120000 index 00000000000..9a9c6d975a9 --- /dev/null +++ b/docs/ja/operations/settings/query_complexity.md @@ -0,0 +1 @@ +../../../en/operations/settings/query_complexity.md \ No newline at end of file diff --git a/docs/ja/operations/settings/settings.md b/docs/ja/operations/settings/settings.md new file mode 120000 index 00000000000..0c8df3cfc90 --- /dev/null +++ b/docs/ja/operations/settings/settings.md @@ -0,0 +1 @@ +../../../en/operations/settings/settings.md \ No newline at end of file diff --git a/docs/ja/operations/settings/settings_profiles.md b/docs/ja/operations/settings/settings_profiles.md new file mode 120000 index 00000000000..35d9747ad56 --- /dev/null +++ b/docs/ja/operations/settings/settings_profiles.md @@ -0,0 +1 @@ +../../../en/operations/settings/settings_profiles.md \ No newline at end of file diff --git a/docs/ja/operations/settings/settings_users.md b/docs/ja/operations/settings/settings_users.md new file mode 120000 index 00000000000..3a6a7cf6948 --- /dev/null +++ b/docs/ja/operations/settings/settings_users.md @@ -0,0 +1 @@ +../../../en/operations/settings/settings_users.md \ No newline at end of file diff --git a/docs/ja/operations/system_tables.md b/docs/ja/operations/system_tables.md new file mode 120000 index 00000000000..c5701190dca --- /dev/null +++ b/docs/ja/operations/system_tables.md @@ -0,0 +1 @@ +../../en/operations/system_tables.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/aggregatingmergetree.md b/docs/ja/operations/table_engines/aggregatingmergetree.md new file mode 120000 index 00000000000..907a073e0c8 --- /dev/null +++ b/docs/ja/operations/table_engines/aggregatingmergetree.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/aggregatingmergetree.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/buffer.md b/docs/ja/operations/table_engines/buffer.md new file mode 120000 index 00000000000..0a3c372fa67 --- /dev/null +++ b/docs/ja/operations/table_engines/buffer.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/buffer.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/collapsingmergetree.md b/docs/ja/operations/table_engines/collapsingmergetree.md new file mode 120000 index 00000000000..ef5cebb48d8 --- /dev/null +++ b/docs/ja/operations/table_engines/collapsingmergetree.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/collapsingmergetree.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/custom_partitioning_key.md b/docs/ja/operations/table_engines/custom_partitioning_key.md new file mode 120000 index 00000000000..a9d18cacb25 --- /dev/null +++ b/docs/ja/operations/table_engines/custom_partitioning_key.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/custom_partitioning_key.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/dictionary.md b/docs/ja/operations/table_engines/dictionary.md new file mode 120000 index 00000000000..2a95f4a669b --- /dev/null +++ b/docs/ja/operations/table_engines/dictionary.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/dictionary.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/distributed.md b/docs/ja/operations/table_engines/distributed.md new file mode 120000 index 00000000000..46994303c35 --- /dev/null +++ b/docs/ja/operations/table_engines/distributed.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/distributed.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/external_data.md b/docs/ja/operations/table_engines/external_data.md new file mode 120000 index 00000000000..27a7b6acec2 --- /dev/null +++ b/docs/ja/operations/table_engines/external_data.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/external_data.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/file.md b/docs/ja/operations/table_engines/file.md new file mode 120000 index 00000000000..27dffc8d78f --- /dev/null +++ b/docs/ja/operations/table_engines/file.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/file.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/graphitemergetree.md b/docs/ja/operations/table_engines/graphitemergetree.md new file mode 120000 index 00000000000..654425d050a --- /dev/null +++ b/docs/ja/operations/table_engines/graphitemergetree.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/graphitemergetree.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/hdfs.md b/docs/ja/operations/table_engines/hdfs.md new file mode 120000 index 00000000000..d4dbfa46e68 --- /dev/null +++ b/docs/ja/operations/table_engines/hdfs.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/hdfs.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/index.md b/docs/ja/operations/table_engines/index.md new file mode 120000 index 00000000000..994dff9b516 --- /dev/null +++ b/docs/ja/operations/table_engines/index.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/index.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/jdbc.md b/docs/ja/operations/table_engines/jdbc.md new file mode 120000 index 00000000000..5165d704b9a --- /dev/null +++ b/docs/ja/operations/table_engines/jdbc.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/jdbc.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/join.md b/docs/ja/operations/table_engines/join.md new file mode 120000 index 00000000000..0914ab950ed --- /dev/null +++ b/docs/ja/operations/table_engines/join.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/join.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/kafka.md b/docs/ja/operations/table_engines/kafka.md new file mode 120000 index 00000000000..cb7bd5dd0f8 --- /dev/null +++ b/docs/ja/operations/table_engines/kafka.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/kafka.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/log.md b/docs/ja/operations/table_engines/log.md new file mode 120000 index 00000000000..2c39ba68522 --- /dev/null +++ b/docs/ja/operations/table_engines/log.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/log.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/log_family.md b/docs/ja/operations/table_engines/log_family.md new file mode 120000 index 00000000000..8c5b5f0365b --- /dev/null +++ b/docs/ja/operations/table_engines/log_family.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/log_family.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/materializedview.md b/docs/ja/operations/table_engines/materializedview.md new file mode 120000 index 00000000000..e3b5deb73dc --- /dev/null +++ b/docs/ja/operations/table_engines/materializedview.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/materializedview.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/memory.md b/docs/ja/operations/table_engines/memory.md new file mode 120000 index 00000000000..eee940c7bd3 --- /dev/null +++ b/docs/ja/operations/table_engines/memory.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/memory.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/merge.md b/docs/ja/operations/table_engines/merge.md new file mode 120000 index 00000000000..9e17d9bb939 --- /dev/null +++ b/docs/ja/operations/table_engines/merge.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/merge.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/mergetree.md b/docs/ja/operations/table_engines/mergetree.md new file mode 120000 index 00000000000..cc6ac1e5297 --- /dev/null +++ b/docs/ja/operations/table_engines/mergetree.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/mergetree.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/mysql.md b/docs/ja/operations/table_engines/mysql.md new file mode 120000 index 00000000000..e4c268658cf --- /dev/null +++ b/docs/ja/operations/table_engines/mysql.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/mysql.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/null.md b/docs/ja/operations/table_engines/null.md new file mode 120000 index 00000000000..c7d9264571e --- /dev/null +++ b/docs/ja/operations/table_engines/null.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/null.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/odbc.md b/docs/ja/operations/table_engines/odbc.md new file mode 120000 index 00000000000..06091fd5377 --- /dev/null +++ b/docs/ja/operations/table_engines/odbc.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/odbc.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/replacingmergetree.md b/docs/ja/operations/table_engines/replacingmergetree.md new file mode 120000 index 00000000000..63ff25a4dd6 --- /dev/null +++ b/docs/ja/operations/table_engines/replacingmergetree.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/replacingmergetree.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/replication.md b/docs/ja/operations/table_engines/replication.md new file mode 120000 index 00000000000..b4b22ac708b --- /dev/null +++ b/docs/ja/operations/table_engines/replication.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/replication.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/set.md b/docs/ja/operations/table_engines/set.md new file mode 120000 index 00000000000..d37e659badd --- /dev/null +++ b/docs/ja/operations/table_engines/set.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/set.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/stripelog.md b/docs/ja/operations/table_engines/stripelog.md new file mode 120000 index 00000000000..f6521a41e3e --- /dev/null +++ b/docs/ja/operations/table_engines/stripelog.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/stripelog.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/summingmergetree.md b/docs/ja/operations/table_engines/summingmergetree.md new file mode 120000 index 00000000000..2b67e953d8a --- /dev/null +++ b/docs/ja/operations/table_engines/summingmergetree.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/summingmergetree.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/tinylog.md b/docs/ja/operations/table_engines/tinylog.md new file mode 120000 index 00000000000..bda90c7d5ce --- /dev/null +++ b/docs/ja/operations/table_engines/tinylog.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/tinylog.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/url.md b/docs/ja/operations/table_engines/url.md new file mode 120000 index 00000000000..d0de71dcf40 --- /dev/null +++ b/docs/ja/operations/table_engines/url.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/url.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/versionedcollapsingmergetree.md b/docs/ja/operations/table_engines/versionedcollapsingmergetree.md new file mode 120000 index 00000000000..5843fba70b8 --- /dev/null +++ b/docs/ja/operations/table_engines/versionedcollapsingmergetree.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/versionedcollapsingmergetree.md \ No newline at end of file diff --git a/docs/ja/operations/table_engines/view.md b/docs/ja/operations/table_engines/view.md new file mode 120000 index 00000000000..3f2164181a7 --- /dev/null +++ b/docs/ja/operations/table_engines/view.md @@ -0,0 +1 @@ +../../../en/operations/table_engines/view.md \ No newline at end of file diff --git a/docs/ja/operations/tips.md b/docs/ja/operations/tips.md new file mode 120000 index 00000000000..9b3413bdbc3 --- /dev/null +++ b/docs/ja/operations/tips.md @@ -0,0 +1 @@ +../../en/operations/tips.md \ No newline at end of file diff --git a/docs/ja/operations/troubleshooting.md b/docs/ja/operations/troubleshooting.md new file mode 120000 index 00000000000..84f0ff34f41 --- /dev/null +++ b/docs/ja/operations/troubleshooting.md @@ -0,0 +1 @@ +../../en/operations/troubleshooting.md \ No newline at end of file diff --git a/docs/ja/operations/update.md b/docs/ja/operations/update.md new file mode 120000 index 00000000000..88a092c0dff --- /dev/null +++ b/docs/ja/operations/update.md @@ -0,0 +1 @@ +../../en/operations/update.md \ No newline at end of file diff --git a/docs/ja/operations/utils/clickhouse-copier.md b/docs/ja/operations/utils/clickhouse-copier.md new file mode 120000 index 00000000000..c9e89e33c7b --- /dev/null +++ b/docs/ja/operations/utils/clickhouse-copier.md @@ -0,0 +1 @@ +../../../en/operations/utils/clickhouse-copier.md \ No newline at end of file diff --git a/docs/ja/operations/utils/clickhouse-local.md b/docs/ja/operations/utils/clickhouse-local.md new file mode 120000 index 00000000000..032aaaa2b84 --- /dev/null +++ b/docs/ja/operations/utils/clickhouse-local.md @@ -0,0 +1 @@ +../../../en/operations/utils/clickhouse-local.md \ No newline at end of file diff --git a/docs/ja/operations/utils/index.md b/docs/ja/operations/utils/index.md new file mode 120000 index 00000000000..dd089d1ef4b --- /dev/null +++ b/docs/ja/operations/utils/index.md @@ -0,0 +1 @@ +../../../en/operations/utils/index.md \ No newline at end of file diff --git a/docs/ja/query_language/agg_functions/combinators.md b/docs/ja/query_language/agg_functions/combinators.md new file mode 120000 index 00000000000..2b914cebd15 --- /dev/null +++ b/docs/ja/query_language/agg_functions/combinators.md @@ -0,0 +1 @@ +../../../en/query_language/agg_functions/combinators.md \ No newline at end of file diff --git a/docs/ja/query_language/agg_functions/index.md b/docs/ja/query_language/agg_functions/index.md new file mode 120000 index 00000000000..2fcf67abdeb --- /dev/null +++ b/docs/ja/query_language/agg_functions/index.md @@ -0,0 +1 @@ +../../../en/query_language/agg_functions/index.md \ No newline at end of file diff --git a/docs/ja/query_language/agg_functions/parametric_functions.md b/docs/ja/query_language/agg_functions/parametric_functions.md new file mode 120000 index 00000000000..fd3ffafcc5b --- /dev/null +++ b/docs/ja/query_language/agg_functions/parametric_functions.md @@ -0,0 +1 @@ +../../../en/query_language/agg_functions/parametric_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/agg_functions/reference.md b/docs/ja/query_language/agg_functions/reference.md new file mode 120000 index 00000000000..c5651cb0793 --- /dev/null +++ b/docs/ja/query_language/agg_functions/reference.md @@ -0,0 +1 @@ +../../../en/query_language/agg_functions/reference.md \ No newline at end of file diff --git a/docs/ja/query_language/alter.md b/docs/ja/query_language/alter.md new file mode 120000 index 00000000000..44f4ecf9737 --- /dev/null +++ b/docs/ja/query_language/alter.md @@ -0,0 +1 @@ +../../en/query_language/alter.md \ No newline at end of file diff --git a/docs/ja/query_language/create.md b/docs/ja/query_language/create.md new file mode 120000 index 00000000000..a13304d176e --- /dev/null +++ b/docs/ja/query_language/create.md @@ -0,0 +1 @@ +../../en/query_language/create.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/external_dicts.md b/docs/ja/query_language/dicts/external_dicts.md new file mode 120000 index 00000000000..491b94bffe6 --- /dev/null +++ b/docs/ja/query_language/dicts/external_dicts.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/external_dicts.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/external_dicts_dict.md b/docs/ja/query_language/dicts/external_dicts_dict.md new file mode 120000 index 00000000000..e27820fee60 --- /dev/null +++ b/docs/ja/query_language/dicts/external_dicts_dict.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/external_dicts_dict.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/external_dicts_dict_layout.md b/docs/ja/query_language/dicts/external_dicts_dict_layout.md new file mode 120000 index 00000000000..e391c5be723 --- /dev/null +++ b/docs/ja/query_language/dicts/external_dicts_dict_layout.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/external_dicts_dict_layout.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/external_dicts_dict_lifetime.md b/docs/ja/query_language/dicts/external_dicts_dict_lifetime.md new file mode 120000 index 00000000000..03b53c09077 --- /dev/null +++ b/docs/ja/query_language/dicts/external_dicts_dict_lifetime.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/external_dicts_dict_lifetime.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/external_dicts_dict_sources.md b/docs/ja/query_language/dicts/external_dicts_dict_sources.md new file mode 120000 index 00000000000..d4f4bf8ef3e --- /dev/null +++ b/docs/ja/query_language/dicts/external_dicts_dict_sources.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/external_dicts_dict_sources.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/external_dicts_dict_structure.md b/docs/ja/query_language/dicts/external_dicts_dict_structure.md new file mode 120000 index 00000000000..69ff759caea --- /dev/null +++ b/docs/ja/query_language/dicts/external_dicts_dict_structure.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/external_dicts_dict_structure.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/index.md b/docs/ja/query_language/dicts/index.md new file mode 120000 index 00000000000..fdc188ca2a2 --- /dev/null +++ b/docs/ja/query_language/dicts/index.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/index.md \ No newline at end of file diff --git a/docs/ja/query_language/dicts/internal_dicts.md b/docs/ja/query_language/dicts/internal_dicts.md new file mode 120000 index 00000000000..3f9408dcd45 --- /dev/null +++ b/docs/ja/query_language/dicts/internal_dicts.md @@ -0,0 +1 @@ +../../../en/query_language/dicts/internal_dicts.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/arithmetic_functions.md b/docs/ja/query_language/functions/arithmetic_functions.md new file mode 120000 index 00000000000..c22acb8c7f5 --- /dev/null +++ b/docs/ja/query_language/functions/arithmetic_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/arithmetic_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/array_functions.md b/docs/ja/query_language/functions/array_functions.md new file mode 120000 index 00000000000..268b2295a97 --- /dev/null +++ b/docs/ja/query_language/functions/array_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/array_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/array_join.md b/docs/ja/query_language/functions/array_join.md new file mode 120000 index 00000000000..b100dac784d --- /dev/null +++ b/docs/ja/query_language/functions/array_join.md @@ -0,0 +1 @@ +../../../en/query_language/functions/array_join.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/bit_functions.md b/docs/ja/query_language/functions/bit_functions.md new file mode 120000 index 00000000000..b5cccd0c56c --- /dev/null +++ b/docs/ja/query_language/functions/bit_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/bit_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/bitmap_functions.md b/docs/ja/query_language/functions/bitmap_functions.md new file mode 120000 index 00000000000..0a31d3d71d8 --- /dev/null +++ b/docs/ja/query_language/functions/bitmap_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/bitmap_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/comparison_functions.md b/docs/ja/query_language/functions/comparison_functions.md new file mode 120000 index 00000000000..417c589867c --- /dev/null +++ b/docs/ja/query_language/functions/comparison_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/comparison_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/conditional_functions.md b/docs/ja/query_language/functions/conditional_functions.md new file mode 120000 index 00000000000..ad0d775dbb5 --- /dev/null +++ b/docs/ja/query_language/functions/conditional_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/conditional_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/date_time_functions.md b/docs/ja/query_language/functions/date_time_functions.md new file mode 120000 index 00000000000..d11b9b8bb6b --- /dev/null +++ b/docs/ja/query_language/functions/date_time_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/date_time_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/encoding_functions.md b/docs/ja/query_language/functions/encoding_functions.md new file mode 120000 index 00000000000..b2e6be1405b --- /dev/null +++ b/docs/ja/query_language/functions/encoding_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/encoding_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/ext_dict_functions.md b/docs/ja/query_language/functions/ext_dict_functions.md new file mode 120000 index 00000000000..6318f900e4b --- /dev/null +++ b/docs/ja/query_language/functions/ext_dict_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/ext_dict_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/functions_for_nulls.md b/docs/ja/query_language/functions/functions_for_nulls.md new file mode 120000 index 00000000000..fa57e10ad15 --- /dev/null +++ b/docs/ja/query_language/functions/functions_for_nulls.md @@ -0,0 +1 @@ +../../../en/query_language/functions/functions_for_nulls.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/geo.md b/docs/ja/query_language/functions/geo.md new file mode 120000 index 00000000000..86fa3a85d34 --- /dev/null +++ b/docs/ja/query_language/functions/geo.md @@ -0,0 +1 @@ +../../../en/query_language/functions/geo.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/hash_functions.md b/docs/ja/query_language/functions/hash_functions.md new file mode 120000 index 00000000000..90de8ba97e7 --- /dev/null +++ b/docs/ja/query_language/functions/hash_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/hash_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/higher_order_functions.md b/docs/ja/query_language/functions/higher_order_functions.md new file mode 120000 index 00000000000..077feba2a3e --- /dev/null +++ b/docs/ja/query_language/functions/higher_order_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/higher_order_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/in_functions.md b/docs/ja/query_language/functions/in_functions.md new file mode 120000 index 00000000000..3ae5f24dbca --- /dev/null +++ b/docs/ja/query_language/functions/in_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/in_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/index.md b/docs/ja/query_language/functions/index.md new file mode 120000 index 00000000000..a4e9d619cc0 --- /dev/null +++ b/docs/ja/query_language/functions/index.md @@ -0,0 +1 @@ +../../../en/query_language/functions/index.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/ip_address_functions.md b/docs/ja/query_language/functions/ip_address_functions.md new file mode 120000 index 00000000000..b58175a7cdf --- /dev/null +++ b/docs/ja/query_language/functions/ip_address_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/ip_address_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/json_functions.md b/docs/ja/query_language/functions/json_functions.md new file mode 120000 index 00000000000..1b37184e006 --- /dev/null +++ b/docs/ja/query_language/functions/json_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/json_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/logical_functions.md b/docs/ja/query_language/functions/logical_functions.md new file mode 120000 index 00000000000..32015440e09 --- /dev/null +++ b/docs/ja/query_language/functions/logical_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/logical_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/machine_learning_functions.md b/docs/ja/query_language/functions/machine_learning_functions.md new file mode 120000 index 00000000000..4509602717e --- /dev/null +++ b/docs/ja/query_language/functions/machine_learning_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/machine_learning_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/math_functions.md b/docs/ja/query_language/functions/math_functions.md new file mode 120000 index 00000000000..e01674eca4d --- /dev/null +++ b/docs/ja/query_language/functions/math_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/math_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/other_functions.md b/docs/ja/query_language/functions/other_functions.md new file mode 120000 index 00000000000..65164784ced --- /dev/null +++ b/docs/ja/query_language/functions/other_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/other_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/random_functions.md b/docs/ja/query_language/functions/random_functions.md new file mode 120000 index 00000000000..b873e0c86ac --- /dev/null +++ b/docs/ja/query_language/functions/random_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/random_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/rounding_functions.md b/docs/ja/query_language/functions/rounding_functions.md new file mode 120000 index 00000000000..e1217e3b25a --- /dev/null +++ b/docs/ja/query_language/functions/rounding_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/rounding_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/splitting_merging_functions.md b/docs/ja/query_language/functions/splitting_merging_functions.md new file mode 120000 index 00000000000..5f8771abdec --- /dev/null +++ b/docs/ja/query_language/functions/splitting_merging_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/splitting_merging_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/string_functions.md b/docs/ja/query_language/functions/string_functions.md new file mode 120000 index 00000000000..cc4104aaf53 --- /dev/null +++ b/docs/ja/query_language/functions/string_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/string_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/string_replace_functions.md b/docs/ja/query_language/functions/string_replace_functions.md new file mode 120000 index 00000000000..4ec963ffd0f --- /dev/null +++ b/docs/ja/query_language/functions/string_replace_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/string_replace_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/string_search_functions.md b/docs/ja/query_language/functions/string_search_functions.md new file mode 120000 index 00000000000..0a2c7f4c4f1 --- /dev/null +++ b/docs/ja/query_language/functions/string_search_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/string_search_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/type_conversion_functions.md b/docs/ja/query_language/functions/type_conversion_functions.md new file mode 120000 index 00000000000..fcf51570d15 --- /dev/null +++ b/docs/ja/query_language/functions/type_conversion_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/type_conversion_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/url_functions.md b/docs/ja/query_language/functions/url_functions.md new file mode 120000 index 00000000000..529e4ffdd53 --- /dev/null +++ b/docs/ja/query_language/functions/url_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/url_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/uuid_functions.md b/docs/ja/query_language/functions/uuid_functions.md new file mode 120000 index 00000000000..95e3ded0477 --- /dev/null +++ b/docs/ja/query_language/functions/uuid_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/uuid_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/ym_dict_functions.md b/docs/ja/query_language/functions/ym_dict_functions.md new file mode 120000 index 00000000000..ec5ddc84479 --- /dev/null +++ b/docs/ja/query_language/functions/ym_dict_functions.md @@ -0,0 +1 @@ +../../../en/query_language/functions/ym_dict_functions.md \ No newline at end of file diff --git a/docs/ja/query_language/index.md b/docs/ja/query_language/index.md new file mode 120000 index 00000000000..44dfff9bb18 --- /dev/null +++ b/docs/ja/query_language/index.md @@ -0,0 +1 @@ +../../en/query_language/index.md \ No newline at end of file diff --git a/docs/ja/query_language/insert_into.md b/docs/ja/query_language/insert_into.md new file mode 120000 index 00000000000..29b47662b0d --- /dev/null +++ b/docs/ja/query_language/insert_into.md @@ -0,0 +1 @@ +../../en/query_language/insert_into.md \ No newline at end of file diff --git a/docs/ja/query_language/misc.md b/docs/ja/query_language/misc.md new file mode 120000 index 00000000000..3bd814f3568 --- /dev/null +++ b/docs/ja/query_language/misc.md @@ -0,0 +1 @@ +../../en/query_language/misc.md \ No newline at end of file diff --git a/docs/ja/query_language/operators.md b/docs/ja/query_language/operators.md new file mode 120000 index 00000000000..f94df928a82 --- /dev/null +++ b/docs/ja/query_language/operators.md @@ -0,0 +1 @@ +../../en/query_language/operators.md \ No newline at end of file diff --git a/docs/ja/query_language/select.md b/docs/ja/query_language/select.md new file mode 120000 index 00000000000..c8ec8369383 --- /dev/null +++ b/docs/ja/query_language/select.md @@ -0,0 +1 @@ +../../en/query_language/select.md \ No newline at end of file diff --git a/docs/ja/query_language/show.md b/docs/ja/query_language/show.md new file mode 120000 index 00000000000..4c2f4cf2c4f --- /dev/null +++ b/docs/ja/query_language/show.md @@ -0,0 +1 @@ +../../en/query_language/show.md \ No newline at end of file diff --git a/docs/ja/query_language/syntax.md b/docs/ja/query_language/syntax.md new file mode 120000 index 00000000000..5307fd51ae8 --- /dev/null +++ b/docs/ja/query_language/syntax.md @@ -0,0 +1 @@ +../../en/query_language/syntax.md \ No newline at end of file diff --git a/docs/ja/query_language/system.md b/docs/ja/query_language/system.md new file mode 120000 index 00000000000..6061858c3f2 --- /dev/null +++ b/docs/ja/query_language/system.md @@ -0,0 +1 @@ +../../en/query_language/system.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/file.md b/docs/ja/query_language/table_functions/file.md new file mode 120000 index 00000000000..a514547109a --- /dev/null +++ b/docs/ja/query_language/table_functions/file.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/file.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/hdfs.md b/docs/ja/query_language/table_functions/hdfs.md new file mode 120000 index 00000000000..2616e737eb6 --- /dev/null +++ b/docs/ja/query_language/table_functions/hdfs.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/hdfs.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/index.md b/docs/ja/query_language/table_functions/index.md new file mode 120000 index 00000000000..89b22522859 --- /dev/null +++ b/docs/ja/query_language/table_functions/index.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/index.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/input.md b/docs/ja/query_language/table_functions/input.md new file mode 120000 index 00000000000..f23cc8ee673 --- /dev/null +++ b/docs/ja/query_language/table_functions/input.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/input.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/jdbc.md b/docs/ja/query_language/table_functions/jdbc.md new file mode 120000 index 00000000000..73bec80ca58 --- /dev/null +++ b/docs/ja/query_language/table_functions/jdbc.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/jdbc.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/merge.md b/docs/ja/query_language/table_functions/merge.md new file mode 120000 index 00000000000..383f6c88331 --- /dev/null +++ b/docs/ja/query_language/table_functions/merge.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/merge.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/mysql.md b/docs/ja/query_language/table_functions/mysql.md new file mode 120000 index 00000000000..75c032cc63f --- /dev/null +++ b/docs/ja/query_language/table_functions/mysql.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/mysql.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/numbers.md b/docs/ja/query_language/table_functions/numbers.md new file mode 120000 index 00000000000..a679b915669 --- /dev/null +++ b/docs/ja/query_language/table_functions/numbers.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/numbers.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/odbc.md b/docs/ja/query_language/table_functions/odbc.md new file mode 120000 index 00000000000..7620f920494 --- /dev/null +++ b/docs/ja/query_language/table_functions/odbc.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/odbc.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/remote.md b/docs/ja/query_language/table_functions/remote.md new file mode 120000 index 00000000000..b157c4076d3 --- /dev/null +++ b/docs/ja/query_language/table_functions/remote.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/remote.md \ No newline at end of file diff --git a/docs/ja/query_language/table_functions/url.md b/docs/ja/query_language/table_functions/url.md new file mode 120000 index 00000000000..038e08f7ba9 --- /dev/null +++ b/docs/ja/query_language/table_functions/url.md @@ -0,0 +1 @@ +../../../en/query_language/table_functions/url.md \ No newline at end of file diff --git a/docs/ja/roadmap.md b/docs/ja/roadmap.md new file mode 120000 index 00000000000..24df86352b3 --- /dev/null +++ b/docs/ja/roadmap.md @@ -0,0 +1 @@ +../en/roadmap.md \ No newline at end of file diff --git a/docs/ja/security_changelog.md b/docs/ja/security_changelog.md new file mode 120000 index 00000000000..101a4f4e48c --- /dev/null +++ b/docs/ja/security_changelog.md @@ -0,0 +1 @@ +../en/security_changelog.md \ No newline at end of file diff --git a/docs/redirects.txt b/docs/redirects.txt index 0ff077b660c..b38f6d242f2 100644 --- a/docs/redirects.txt +++ b/docs/redirects.txt @@ -1,3 +1,4 @@ +introduction/ya_metrika_task.md introduction/history.md system_tables.md operations/system_tables.md system_tables/system.asynchronous_metrics.md operations/system_tables.md system_tables/system.clusters.md operations/system_tables.md diff --git a/docs/ru/getting_started/index.md b/docs/ru/getting_started/index.md index c03ac58f24b..a8d0fbaa5b1 100644 --- a/docs/ru/getting_started/index.md +++ b/docs/ru/getting_started/index.md @@ -1,138 +1,10 @@ # Начало работы -## Системные требования +Если вы новичок в ClickHouse и хотите получить вживую оценить его производительность, прежде всего нужно пройти через [процесс установки](install.md). -ClickHouse может работать на любом Linux, FreeBSD или Mac OS X с архитектурой процессора x86\_64. +После этого можно выбрать один из следующих вариантов: -Хотя предсобранные релизы обычно компилируются с использованием набора инструкций SSE 4.2, что добавляет использование поддерживающего его процессора в список системных требований. Команда для проверки наличия поддержки инструкций SSE 4.2 на текущем процессоре: - -```bash -$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" -``` - -## Установка - -### Из DEB пакетов - -Яндекс рекомендует использовать официальные скомпилированные `deb` пакеты для Debian или Ubuntu. - -Чтобы установить официальные пакеты, пропишите репозиторий Яндекса в `/etc/apt/sources.list` или в отдельный файл `/etc/apt/sources.list.d/clickhouse.list`: - -```bash -$ deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ -``` - -Если вы хотите использовать наиболее свежую тестовую, замените `stable` на `testing` (не рекомендуется для production окружений). - -Затем для самой установки пакетов выполните: - -```bash -$ sudo apt-get install dirmngr # optional -$ sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional -$ sudo apt-get update -$ sudo apt-get install clickhouse-client clickhouse-server -``` - -Также эти пакеты можно скачать и установить вручную отсюда: . - -### Из RPM пакетов - -Команда ClickHouse в Яндексе рекомендует использовать официальные предкомпилированные `rpm` пакеты для CentOS, RedHad и всех остальных дистрибутивов Linux, основанных на rpm. - -Сначала нужно подключить официальный репозиторий: -```bash -$ sudo yum install yum-utils -$ sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG -$ sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 -``` - -Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). - -Then run these commands to actually install packages: -Для, собственно, установки пакетов необходимо выполнить следующие команды: - -```bash -$ sudo yum install clickhouse-server clickhouse-client -``` - -Также есть возможность установить пакеты вручную, скачав отсюда: . - -### Из Docker образа - -Для запуска ClickHouse в Docker нужно следовать инструкции на [Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/). Внутри образов используются официальные `deb` пакеты. - -### Из исходного кода - -Для компиляции ClickHouse вручную, используйте инструкцию для [Linux](../development/build.md) или [Mac OS X](../development/build_osx.md). - -Можно скомпилировать пакеты и установить их, либо использовать программы без установки пакетов. Также при ручной сборке можно отключить необходимость поддержки набора инструкций SSE 4.2 или собрать под процессоры архитектуры AArch64. - -```text -Client: dbms/programs/clickhouse-client -Server: dbms/programs/clickhouse-server -``` - -Для работы собранного вручную сервера необходимо создать директории для данных и метаданных, а также сделать их `chown` для желаемого пользователя. Пути к этим директориям могут быть изменены в конфигурационном файле сервера (src/dbms/programs/server/config.xml), по умолчанию используются следующие: - -```text -/opt/clickhouse/data/default/ -/opt/clickhouse/metadata/default/ -``` - -На Gentoo для установки ClickHouse из исходного кода можно использовать просто `emerge clickhouse`. - -## Запуск - -Для запуска сервера в качестве демона, выполните: - -```bash -$ sudo service clickhouse-server start -``` - -Смотрите логи в директории `/var/log/clickhouse-server/`. - -Если сервер не стартует, проверьте корректность конфигурации в файле `/etc/clickhouse-server/config.xml` - -Также можно запустить сервер вручную из консоли: - -```bash -$ clickhouse-server --config-file=/etc/clickhouse-server/config.xml -``` - -При этом, лог будет выводиться в консоль, что удобно для разработки. -Если конфигурационный файл лежит в текущей директории, то указывать параметр `--config-file` не требуется, по умолчанию будет использован файл `./config.xml`. - -После запуска сервера, соединиться с ним можно с помощью клиента командной строки: - -```bash -$ clickhouse-client -``` - -По умолчанию он соединяется с localhost:9000, от имени пользователя `default` без пароля. Также клиент может быть использован для соединения с удалённым сервером с помощью аргумента `--host`. - -Терминал должен использовать кодировку UTF-8. - -Более подробная информация о клиенте располагается в разделе [«Клиент командной строки»](../interfaces/cli.md). - -Пример проверки работоспособности системы: - -```bash -$ ./clickhouse-client -ClickHouse client version 0.0.18749. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.18749. -``` -```sql -SELECT 1 -``` -```text -┌─1─┐ -│ 1 │ -└───┘ -``` - -**Поздравляем, система работает!** - -Для дальнейших экспериментов можно попробовать загрузить один из тестовых наборов данных или пройти [пошаговое руководство для начинающих](https://clickhouse.yandex/tutorial.html). +* [Пройти подробное руководство для начинающих](tutorial.md) +* [Поэкспериментировать с тестовыми наборами данных](example_datasets/ontime.md) [Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/) diff --git a/docs/ru/getting_started/install.md b/docs/ru/getting_started/install.md new file mode 100644 index 00000000000..29ccd2b14f4 --- /dev/null +++ b/docs/ru/getting_started/install.md @@ -0,0 +1,144 @@ +# Установка + +## Системные требования + +ClickHouse может работать на любой операционной системе Linux, FreeBSD или Mac OS X с архитектурой процессора x86\_64, AArch64 или PowerPC64LE. + +Предварительно собранные пакеты компилируются для x86\_64 и используют набор инструкций SSE 4.2, поэтому, если не указано иное, его поддержка в используемом процессоре, становится дополнительным требованием к системе. Вот команда, чтобы проверить, поддерживает ли текущий процессор SSE 4.2: + +``` bash +$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" +``` + +Чтобы запустить ClickHouse на процессорах, которые не поддерживают SSE 4.2, либо имеют архитектуру AArch64 или PowerPC64LE, необходимо самостоятельно [собрать ClickHouse из исходного кода](#from-sources) с соответствующими настройками конфигурации. + +## Доступные варианты установки + +### Из DEB пакетов {#from-deb-packages} + +Яндекс рекомендует использовать официальные скомпилированные `deb` пакеты для Debian или Ubuntu. + +Чтобы установить официальные пакеты, пропишите репозиторий Яндекса в `/etc/apt/sources.list` или в отдельный файл `/etc/apt/sources.list.d/clickhouse.list`: + +``` +deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ +``` + +Если вы хотите использовать наиболее свежую тестовую, замените `stable` на `testing` (не рекомендуется для production окружений). + +Затем для самой установки пакетов выполните: + +```bash +sudo apt-get install dirmngr # optional +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional +sudo apt-get update +sudo apt-get install clickhouse-client clickhouse-server +``` + +Также эти пакеты можно скачать и установить вручную отсюда: . + +### Из RPM пакетов {#from-rpm-packages} + +Команда ClickHouse в Яндексе рекомендует использовать официальные предкомпилированные `rpm` пакеты для CentOS, RedHad и всех остальных дистрибутивов Linux, основанных на rpm. + +Сначала нужно подключить официальный репозиторий: +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` + +Для использования наиболее свежих версий нужно заменить `stable` на `testing` (рекомендуется для тестовых окружений). + +Then run these commands to actually install packages: +Для, собственно, установки пакетов необходимо выполнить следующие команды: + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +Также есть возможность установить пакеты вручную, скачав отсюда: . + +### Из Docker образа {#from-docker-image} + +Для запуска ClickHouse в Docker нужно следовать инструкции на [Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/). Внутри образов используются официальные `deb` пакеты. + +### Из исходного кода {#from-sources} + +Для компиляции ClickHouse вручную, используйте инструкцию для [Linux](../development/build.md) или [Mac OS X](../development/build_osx.md). + +Можно скомпилировать пакеты и установить их, либо использовать программы без установки пакетов. Также при ручой сборке можно отключить необходимость поддержки набора инструкций SSE 4.2 или собрать под процессоры архитектуры AArch64. + +``` +Client: dbms/programs/clickhouse-client +Server: dbms/programs/clickhouse-server +``` + +Для работы собранного вручную сервера необходимо создать директории для данных и метаданных, а также сделать их `chown` для желаемого пользователя. Пути к этим директориям могут быть изменены в конфигурационном файле сервера (src/dbms/programs/server/config.xml), по умолчанию используются следующие: + +``` +/opt/clickhouse/data/default/ +/opt/clickhouse/metadata/default/ +``` + +На Gentoo для установки ClickHouse из исходного кода можно использовать просто `emerge clickhouse`. + +## Запуск + +Для запуска сервера в качестве демона, выполните: + +``` bash +$ sudo service clickhouse-server start +``` + +Смотрите логи в директории `/var/log/clickhouse-server/`. + +Если сервер не стартует, проверьте корректность конфигурации в файле `/etc/clickhouse-server/config.xml` + +Также можно запустить сервер вручную из консоли: + +``` bash +$ clickhouse-server --config-file=/etc/clickhouse-server/config.xml +``` + +При этом, лог будет выводиться в консоль, что удобно для разработки. +Если конфигурационный файл лежит в текущей директории, то указывать параметр `--config-file` не требуется, по умолчанию будет использован файл `./config.xml`. + +После запуска сервера, соединиться с ним можно с помощью клиента командной строки: + +``` bash +$ clickhouse-client +``` + +По умолчанию он соединяется с localhost:9000, от имени пользователя `default` без пароля. Также клиент может быть использован для соединения с удалённым сервером с помощью аргумента `--host`. + +Терминал должен использовать кодировку UTF-8. + +Более подробная информация о клиенте располагается в разделе [«Клиент командной строки»](../interfaces/cli.md). + +Пример проверки работоспособности системы: + +``` bash +$ ./clickhouse-client +ClickHouse client version 0.0.18749. +Connecting to localhost:9000. +Connected to ClickHouse server version 0.0.18749. + +:) SELECT 1 + +SELECT 1 + +┌─1─┐ +│ 1 │ +└───┘ + +1 rows in set. Elapsed: 0.003 sec. + +:) +``` + +**Поздравляем, система работает!** + +Для дальнейших экспериментов можно попробовать загрузить один из тестовых наборов данных или пройти [пошаговое руководство для начинающих](https://clickhouse.yandex/tutorial.html). + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/getting_started/install/) diff --git a/docs/ru/getting_started/tutorial.md b/docs/ru/getting_started/tutorial.md new file mode 120000 index 00000000000..8bc40816ab2 --- /dev/null +++ b/docs/ru/getting_started/tutorial.md @@ -0,0 +1 @@ +../../en/getting_started/tutorial.md \ No newline at end of file diff --git a/docs/ru/introduction/ya_metrika_task.md b/docs/ru/introduction/history.md similarity index 99% rename from docs/ru/introduction/ya_metrika_task.md rename to docs/ru/introduction/history.md index c7e22346ae5..c0035b51f82 100644 --- a/docs/ru/introduction/ya_metrika_task.md +++ b/docs/ru/introduction/history.md @@ -1,4 +1,4 @@ -# Постановка задачи в Яндекс.Метрике +# История ClickHouse ClickHouse изначально разрабатывался для обеспечения работы [Яндекс.Метрики](https://metrika.yandex.ru/), [второй крупнейшей в мире](http://w3techs.com/technologies/overview/traffic_analysis/all) платформы для веб аналитики, и продолжает быть её ключевым компонентом. При более 13 триллионах записей в базе данных и более 20 миллиардах событий в сутки, ClickHouse позволяет генерировать индивидуально настроенные отчёты на лету напрямую из неагрегированных данных. Данная статья вкратце демонстрирует какие цели исторически стояли перед ClickHouse на ранних этапах его развития. diff --git a/docs/toc_en.yml b/docs/toc_en.yml index 5ed9265b3bc..8a2b32b240a 100644 --- a/docs/toc_en.yml +++ b/docs/toc_en.yml @@ -5,10 +5,12 @@ nav: - 'Distinctive Features of ClickHouse': 'introduction/distinctive_features.md' - 'ClickHouse Features that Can Be Considered Disadvantages': 'introduction/features_considered_disadvantages.md' - 'Performance': 'introduction/performance.md' - - 'The Yandex.Metrica Task': 'introduction/ya_metrika_task.md' + - 'History': 'introduction/history.md' - 'Getting Started': - - 'Deploying and Running': 'getting_started/index.md' + - 'hidden': 'getting_started/index.md' + - 'Installation': 'getting_started/install.md' + - 'Tutorial': 'getting_started/tutorial.md' - 'Example Datasets': - 'OnTime': 'getting_started/example_datasets/ontime.md' - 'New York Taxi Data': 'getting_started/example_datasets/nyc_taxi.md' diff --git a/docs/toc_fa.yml b/docs/toc_fa.yml index 207034a8718..30de03f320a 100644 --- a/docs/toc_fa.yml +++ b/docs/toc_fa.yml @@ -1,15 +1,17 @@ nav: -- 'Introduction': +- 'ﯽﻓﺮﻌﻣ': - 'ClickHouse چیست؟': 'index.md' - ' ویژگی های برجسته ClickHouse': 'introduction/distinctive_features.md' - ' ویژگی های از ClickHouse که می تواند معایبی باشد': 'introduction/features_considered_disadvantages.md' - - 'Performance': 'introduction/performance.md' - - 'The Yandex.Metrica task': 'introduction/ya_metrika_task.md' + - 'ﯽﯾﺍﺭﺎﮐ': 'introduction/performance.md' + - 'ﺦﯾﺭﺎﺗ': 'introduction/history.md' - 'Getting started': - - ' شروع به کار': 'getting_started/index.md' - - 'Example datasets': + - 'hidden': 'getting_started/index.md' + - 'ﯼﺯﺍﺪﻧﺍ ﻩﺍﺭ ﻭ ﺐﺼﻧ': 'getting_started/install.md' + - 'ﺵﺯﻮﻣﺁ': 'getting_started/tutorial.md' + - 'ﻪﻧﻮﻤﻧ ﯼﺎﻫ ﻩﺩﺍﺩ ﻪﻋﻮﻤﺠﻣ': - 'OnTime': 'getting_started/example_datasets/ontime.md' - ' داده های تاکسی New York': 'getting_started/example_datasets/nyc_taxi.md' - ' بنچمارک AMPLab Big Data': 'getting_started/example_datasets/amplab_benchmark.md' @@ -18,7 +20,7 @@ nav: - ' بنچمارک Star Schema': 'getting_started/example_datasets/star_schema.md' - 'Yandex.Metrica Data': 'getting_started/example_datasets/metrica.md' -- 'Interfaces': +- 'ﻂﺑﺍﺭ': - 'Interface ها': 'interfaces/index.md' - ' کلاینت Command-line': 'interfaces/cli.md' - 'Native interface (TCP)': 'interfaces/tcp.md' @@ -33,7 +35,7 @@ nav: - 'رابط های بصری': 'interfaces/third-party/gui.md' - 'پروکسی': 'interfaces/third-party/proxy.md' -- 'Data types': +- 'ﻩﺩﺍﺩ ﻉﺍﻮﻧﺍ': - 'Introduction': 'data_types/index.md' - 'UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64': 'data_types/int_uint.md' - 'Float32, Float64': 'data_types/float.md' diff --git a/docs/toc_ja.yml b/docs/toc_ja.yml index 3c4a5506a06..d96963953fe 100644 --- a/docs/toc_ja.yml +++ b/docs/toc_ja.yml @@ -5,10 +5,12 @@ nav: - 'Distinctive Features of ClickHouse': 'introduction/distinctive_features.md' - 'ClickHouse Features that Can Be Considered Disadvantages': 'introduction/features_considered_disadvantages.md' - 'Performance': 'introduction/performance.md' - - 'The Yandex.Metrica Task': 'introduction/ya_metrika_task.md' + - 'History': 'introduction/history.md' - 'Getting Started': - - 'Deploying and Running': 'getting_started/index.md' + - 'hidden': 'getting_started/index.md' + - 'Installation': 'getting_started/install.md' + - 'Tutorial': 'getting_started/tutorial.md' - 'Example Datasets': - 'OnTime': 'getting_started/example_datasets/ontime.md' - 'New York Taxi Data': 'getting_started/example_datasets/nyc_taxi.md' @@ -215,8 +217,9 @@ nav: - 'Overview of ClickHouse Architecture': 'development/architecture.md' - 'How to Build ClickHouse on Linux': 'development/build.md' - 'How to Build ClickHouse on Mac OS X': 'development/build_osx.md' - - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross.md' - - 'How to Write C++ code': 'development/style.md' + - 'How to Build ClickHouse on Linux for Mac OS X': 'development/build_cross_osx.md' + - 'How to Build ClickHouse on Linux for AARCH64 (ARM64)': 'development/build_cross_arm.md' + - 'How to Write C++ Code': 'development/style.md' - 'How to Run ClickHouse Tests': 'development/tests.md' - 'The Beginner ClickHouse Developer Instruction': 'development/developer_instruction.md' - 'Third-Party Libraries Used': 'development/contrib.md' diff --git a/docs/toc_ru.yml b/docs/toc_ru.yml index 5c71894c8f6..9e4b948cc02 100644 --- a/docs/toc_ru.yml +++ b/docs/toc_ru.yml @@ -5,11 +5,13 @@ nav: - 'Отличительные возможности ClickHouse': 'introduction/distinctive_features.md' - 'Особенности ClickHouse, которые могут считаться недостатками': 'introduction/features_considered_disadvantages.md' - 'Производительность': 'introduction/performance.md' - - 'Постановка задачи в Яндекс.Метрике': 'introduction/ya_metrika_task.md' + - 'История': 'introduction/history.md' - 'Информационная поддержка': 'introduction/info.md' - 'Начало работы': - - 'Установка и запуск': 'getting_started/index.md' + - 'hidden': 'getting_started/index.md' + - 'Установка': 'getting_started/install.md' + - 'Руководство для начинающих': 'getting_started/tutorial.md' - 'Тестовые наборы данных': - 'OnTime': 'getting_started/example_datasets/ontime.md' - 'Данные о такси в Нью-Йорке': 'getting_started/example_datasets/nyc_taxi.md' @@ -214,7 +216,7 @@ nav: - 'Обзор архитектуры ClickHouse': 'development/architecture.md' - 'Как собрать ClickHouse на Linux': 'development/build.md' - 'Как собрать ClickHouse на Mac OS X': 'development/build_osx.md' - - 'Как собрать ClickHouse на Linux для Mac OS X': 'development/build_cross.md' + - 'Как собрать ClickHouse на Linux для Mac OS X': 'development/build_cross_osx.md' - 'Как писать код на C++': 'development/style.md' - 'Как запустить тесты': 'development/tests.md' - 'Инструкция для начинающего разработчика ClickHouse': 'development/developer_instruction.md' diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index c8be2ad7606..3f19fe3f72b 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -5,10 +5,12 @@ nav: - 'ClickHouse的独特功能': 'introduction/distinctive_features.md' - 'ClickHouse功能可被视为缺点': 'introduction/features_considered_disadvantages.md' - '性能': 'introduction/performance.md' - - 'Yandex.Metrica使用案例': 'introduction/ya_metrika_task.md' + - '历史': 'introduction/history.md' - '入门指南': - - '部署运行': 'getting_started/index.md' + - 'hidden': 'getting_started/index.md' + - '安装': 'getting_started/install.md' + - '教程': 'getting_started/tutorial.md' - '示例数据集': - '航班飞行数据': 'getting_started/example_datasets/ontime.md' - '纽约市出租车数据': 'getting_started/example_datasets/nyc_taxi.md' @@ -16,6 +18,7 @@ nav: - '维基访问数据': 'getting_started/example_datasets/wikistat.md' - 'Criteo TB级别点击日志': 'getting_started/example_datasets/criteo.md' - 'Star Schema基准测试': 'getting_started/example_datasets/star_schema.md' + - 'Yandex.Metrica': 'getting_started/example_datasets/metrica.md' - '客户端': - '介绍': 'interfaces/index.md' @@ -212,7 +215,7 @@ nav: - 'ClickHouse架构概述': 'development/architecture.md' - '如何在Linux中编译ClickHouse': 'development/build.md' - '如何在Mac OS X中编译ClickHouse': 'development/build_osx.md' - - '如何在Linux中编译Mac OS X ClickHouse': 'development/build_cross.md' + - '如何在Linux中编译Mac OS X ClickHouse': 'development/build_cross_osx.md' - '如何编写C++代码': 'development/style.md' - '如何运行ClickHouse测试': 'development/tests.md' - '开发者指南': 'development/developer_instruction.md' diff --git a/docs/tools/make_links.sh b/docs/tools/make_links.sh index 084f8b9d97b..04c51424ec8 100755 --- a/docs/tools/make_links.sh +++ b/docs/tools/make_links.sh @@ -11,7 +11,7 @@ function do_make_links() for lang in "${langs[@]}" do # replacing "/./" with / - dst_file="../${lang}/${src_file}" + dst_file="../${lang}${src_file}" dst_file="${dst_file/\/\.\//\/}" mkdir -p $(dirname "${dst_file}") diff --git a/docs/tools/mkdocs-material-theme/assets/flags/ja.svg b/docs/tools/mkdocs-material-theme/assets/flags/ja.svg index 177d0e78819..a666c272523 100644 --- a/docs/tools/mkdocs-material-theme/assets/flags/ja.svg +++ b/docs/tools/mkdocs-material-theme/assets/flags/ja.svg @@ -1,5 +1,8 @@ - - - - + + + + + + + diff --git a/docs/tools/mkdocs-material-theme/partials/language/ja.html b/docs/tools/mkdocs-material-theme/partials/language/ja.html index 09c3b291fed..47341ab06ee 100644 --- a/docs/tools/mkdocs-material-theme/partials/language/ja.html +++ b/docs/tools/mkdocs-material-theme/partials/language/ja.html @@ -7,6 +7,12 @@ "footer.next": "次", "meta.comments": "コメント", "meta.source": "ソース", + "nav.latest": "master", + "nav.multi_page": "マルチページ版", + "nav.pdf": "PDF版", + "nav.release": "リリース", + "nav.single_page": "シングルページ版", + "nav.source": "ClickHouseソースコード", "search.placeholder": "検索", "search.result.placeholder": "検索キーワードを入力してください", "search.result.none": "何も見つかりませんでした", diff --git a/docs/zh/getting_started/example_datasets/metrica.md b/docs/zh/getting_started/example_datasets/metrica.md new file mode 120000 index 00000000000..984023973eb --- /dev/null +++ b/docs/zh/getting_started/example_datasets/metrica.md @@ -0,0 +1 @@ +../../../en/getting_started/example_datasets/metrica.md \ No newline at end of file diff --git a/docs/zh/getting_started/index.md b/docs/zh/getting_started/index.md index b1c94600da0..c73181a6068 100644 --- a/docs/zh/getting_started/index.md +++ b/docs/zh/getting_started/index.md @@ -1,154 +1,10 @@ -# 入门指南 +# 入门 -## 系统要求 +如果您是ClickHouse的新手,并希望亲身体验它的性能,首先您需要通过 [安装过程](install.md). -如果从官方仓库安装,需要确保您使用的是x86\_64处理器构架的Linux并且支持SSE 4.2指令集 +之后,您可以选择以下选项之一: -检查是否支持SSE 4.2: +* [通过详细的教程](tutorial.md) +* [试验示例数据集](example_datasets/ontime.md) -```bash -grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" -``` - -我们推荐使用Ubuntu或者Debian。终端必须使用UTF-8编码。 - -基于rpm的系统,你可以使用第三方的安装包:https://packagecloud.io/altinity/clickhouse 或者直接安装debian安装包。 - -ClickHouse还可以在FreeBSD与Mac OS X上工作。同时它可以在不支持SSE 4.2的x86\_64构架和AArch64 CPUs上编译。 - -## 安装 - -### 为Debian/Ubuntu安装 - -在`/etc/apt/sources.list` (或创建`/etc/apt/sources.list.d/clickhouse.list`文件)中添加仓库: - -```bash -$ deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ -``` - -如果你想使用最新的测试版本,请使用'testing'替换'stable'。 - -然后运行: - -```bash -$ sudo apt-get install dirmngr # optional -$ sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional -$ sudo apt-get update -$ sudo apt-get install clickhouse-client clickhouse-server -``` - -你也可以从这里手动下载安装包:。 - -ClickHouse包含访问控制配置,它们位于`users.xml`文件中(与'config.xml'同目录)。 -默认情况下,允许从任何地方使用默认的‘default’用户无密码的访问ClickHouse。参考‘user/default/networks’。 -有关更多信息,请参考"Configuration files"部分。 - -###为CentOS/RedHat安装 - -Yandex ClickHouse团队建议使用官方预编译的`rpm`软件包,用于CentOS,RedHat和所有其他基于rpm的Linux发行版。 - -首先,您需要添加官方存储库: - -```bash -$ sudo yum install yum-utils -$ sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG -$ sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 -``` - -如果您想使用最新版本,请将`stable`替换为`testing`(建议您在测试环境中使用)。 - -然后运行这些命令以实际安装包: - -```bash -$ sudo yum install clickhouse-server clickhouse-client -``` - -您也可以从此处手动下载和安装软件包:。 - -###使用Docker安装 - -要在Docker中运行ClickHouse,请遵循[Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/)上的指南。这些镜像使用官方的`deb`包构建。 - -### 使用源码安装 - -具体编译方式可以参考build.md。 - -你可以编译并安装它们。 -你也可以直接使用而不进行安装。 - -```text -Client: dbms/programs/clickhouse-client -Server: dbms/programs/clickhouse-server -``` - -在服务器中为数据创建如下目录: - -```text -/opt/clickhouse/data/default/ -/opt/clickhouse/metadata/default/ -``` - -(它们可以在server config中配置。) -为需要的用户运行‘chown’ - -日志的路径可以在server config (src/dbms/programs/server/config.xml)中配置。 - -## 启动 - -可以运行如下命令在后台启动服务: - -```bash -sudo service clickhouse-server start -``` - -可以在`/var/log/clickhouse-server/`目录中查看日志。 - -如果服务没有启动,请检查配置文件 `/etc/clickhouse-server/config.xml`。 - -你也可以在控制台中直接启动服务: - -```bash -clickhouse-server --config-file=/etc/clickhouse-server/config.xml -``` - -在这种情况下,日志将被打印到控制台中,这在开发过程中很方便。 -如果配置文件在当前目录中,你可以不指定‘--config-file’参数。它默认使用‘./config.xml’。 - -你可以使用命令行客户端连接到服务: - -```bash -clickhouse-client -``` - -默认情况下它使用‘default’用户无密码的与localhost:9000服务建立连接。 -客户端也可以用于连接远程服务,例如: - -```bash -clickhouse-client --host=example.com -``` - -有关更多信息,请参考"Command-line client"部分。 - -检查系统是否工作: - -```bash -milovidov@hostname:~/work/metrica/src/dbms/src/Client$ ./clickhouse-client -ClickHouse client version 0.0.18749. -Connecting to localhost:9000. -Connected to ClickHouse server version 0.0.18749. -``` -```sql -:) SELECT 1 -``` -```text -┌─1─┐ -│ 1 │ -└───┘ -``` - -**恭喜,系统已经工作了!** - -为了继续进行实验,你可以尝试下载测试数据集。 - - -[Original article](https://clickhouse.yandex/docs/en/getting_started/) +[来源文章](https://clickhouse.yandex/docs/zh/getting_started/) diff --git a/docs/zh/getting_started/install.md b/docs/zh/getting_started/install.md new file mode 100644 index 00000000000..aa3cb816218 --- /dev/null +++ b/docs/zh/getting_started/install.md @@ -0,0 +1,152 @@ +## 系统要求 + +ClickHouse可以在任何具有x86\_64,AArch64或PowerPC64LE CPU架构的Linux,FreeBSD或Mac OS X上运行。 + +虽然预构建的二进制文件通常是为x86 \ _64编译并利用SSE 4.2指令集,但除非另有说明,否则使用支持它的CPU将成为额外的系统要求。这是检查当前CPU是否支持SSE 4.2的命令: + +``` bash +$ grep -q sse4_2 /proc/cpuinfo && echo "SSE 4.2 supported" || echo "SSE 4.2 not supported" +``` + +要在不支持SSE 4.2或具有AArch64或PowerPC64LE体系结构的处理器上运行ClickHouse,您应该[通过源构建ClickHouse](#from-sources)进行适当的配置调整。 + +##可用的安装选项 + +### 为Debian/Ubuntu安装 {#from-deb-packages} + +在`/etc/apt/sources.list` (或创建`/etc/apt/sources.list.d/clickhouse.list`文件)中添加仓库: + +```text +deb http://repo.yandex.ru/clickhouse/deb/stable/ main/ +``` + +如果你想使用最新的测试版本,请使用'testing'替换'stable'。 + +然后运行: + +```bash +sudo apt-get install dirmngr # optional +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv E0C56BD4 # optional +sudo apt-get update +sudo apt-get install clickhouse-client clickhouse-server +``` + +你也可以从这里手动下载安装包:。 + +ClickHouse包含访问控制配置,它们位于`users.xml`文件中(与'config.xml'同目录)。 +默认情况下,允许从任何地方使用默认的‘default’用户无密码的访问ClickHouse。参考‘user/default/networks’。 +有关更多信息,请参考"Configuration files"部分。 + +###来自RPM包 {#from-rpm-packages} + +Yandex ClickHouse团队建议使用官方预编译的`rpm`软件包,用于CentOS,RedHat和所有其他基于rpm的Linux发行版。 + +首先,您需要添加官方存储库: + +```bash +sudo yum install yum-utils +sudo rpm --import https://repo.yandex.ru/clickhouse/CLICKHOUSE-KEY.GPG +sudo yum-config-manager --add-repo https://repo.yandex.ru/clickhouse/rpm/stable/x86_64 +``` + +如果您想使用最新版本,请将`stable`替换为`testing`(建议您在测试环境中使用)。 + +然后运行这些命令以实际安装包: + +```bash +sudo yum install clickhouse-server clickhouse-client +``` + +您也可以从此处手动下载和安装软件包:。 + +###来自Docker {#from-docker-image} + +要在Docker中运行ClickHouse,请遵循[Docker Hub](https://hub.docker.com/r/yandex/clickhouse-server/)上的指南。那些图像使用官方的`deb`包。 + +### 使用源码安装 {#from-sources} + +具体编译方式可以参考build.md。 + +你可以编译并安装它们。 +你也可以直接使用而不进行安装。 + +```text +Client: dbms/programs/clickhouse-client +Server: dbms/programs/clickhouse-server +``` + +在服务器中为数据创建如下目录: + +```text +/opt/clickhouse/data/default/ +/opt/clickhouse/metadata/default/ +``` + +(它们可以在server config中配置。) +为需要的用户运行‘chown’ + +日志的路径可以在server config (src/dbms/programs/server/config.xml)中配置。 + +## 启动 + +可以运行如下命令在后台启动服务: + +```bash +sudo service clickhouse-server start +``` + +可以在`/var/log/clickhouse-server/`目录中查看日志。 + +如果服务没有启动,请检查配置文件 `/etc/clickhouse-server/config.xml`。 + +你也可以在控制台中直接启动服务: + +```bash +clickhouse-server --config-file=/etc/clickhouse-server/config.xml +``` + +在这种情况下,日志将被打印到控制台中,这在开发过程中很方便。 +如果配置文件在当前目录中,你可以不指定‘--config-file’参数。它默认使用‘./config.xml’。 + +你可以使用命令行客户端连接到服务: + +```bash +clickhouse-client +``` + +默认情况下它使用‘default’用户无密码的与localhost:9000服务建立连接。 +客户端也可以用于连接远程服务,例如: + +```bash +clickhouse-client --host=example.com +``` + +有关更多信息,请参考"Command-line client"部分。 + +检查系统是否工作: + +```bash +milovidov@hostname:~/work/metrica/src/dbms/src/Client$ ./clickhouse-client +ClickHouse client version 0.0.18749. +Connecting to localhost:9000. +Connected to ClickHouse server version 0.0.18749. + +:) SELECT 1 + +SELECT 1 + +┌─1─┐ +│ 1 │ +└───┘ + +1 rows in set. Elapsed: 0.003 sec. + +:) +``` + +**恭喜,系统已经工作了!** + +为了继续进行实验,你可以尝试下载测试数据集。 + + +[Original article](https://clickhouse.yandex/docs/en/getting_started/install/) diff --git a/docs/zh/getting_started/tutorial.md b/docs/zh/getting_started/tutorial.md new file mode 120000 index 00000000000..8bc40816ab2 --- /dev/null +++ b/docs/zh/getting_started/tutorial.md @@ -0,0 +1 @@ +../../en/getting_started/tutorial.md \ No newline at end of file diff --git a/docs/zh/introduction/ya_metrika_task.md b/docs/zh/introduction/history.md similarity index 99% rename from docs/zh/introduction/ya_metrika_task.md rename to docs/zh/introduction/history.md index da4b18826e0..86fe02f84d5 100644 --- a/docs/zh/introduction/ya_metrika_task.md +++ b/docs/zh/introduction/history.md @@ -1,4 +1,4 @@ -# Yandex.Metrica的使用案例 +# ClickHouse历史 ClickHouse最初是为 [Yandex.Metrica](https://metrica.yandex.com/) [世界第二大Web分析平台](http://w3techs.com/technologies/overview/traffic_analysis/all) 而开发的。多年来一直作为该系统的核心组件被该系统持续使用着。目前为止,该系统在ClickHouse中有超过13万亿条记录,并且每天超过200多亿个事件被处理。它允许直接从原始数据中动态查询并生成报告。本文简要介绍了ClickHouse在其早期发展阶段的目标。 diff --git a/website/nginx/default.conf b/website/nginx/default.conf index 98edad41055..fc029323fe2 100644 --- a/website/nginx/default.conf +++ b/website/nginx/default.conf @@ -14,6 +14,8 @@ server { } rewrite ^/docs/$ https://clickhouse.yandex/docs/en/ permanent; + rewrite ^/tutorial.html$ https://clickhouse.yandex/docs/en/getting_started/tutorial/ permanent; + rewrite ^/presentations/(.*)$ https://clickhouse.github.io/clickhouse-presentations/$1 permanent; rewrite ^/reference_en.html$ https://clickhouse.yandex/docs/en/single/ permanent; rewrite ^/reference_ru.html$ https://clickhouse.yandex/docs/ru/single/ permanent; rewrite ^/presentations/(.*)$ https://clickhouse.github.io/clickhouse-presentations/$1 permanent; diff --git a/website/tutorial.html b/website/tutorial.html deleted file mode 100644 index 52216f61dc8..00000000000 --- a/website/tutorial.html +++ /dev/null @@ -1,649 +0,0 @@ - - - - - ClickHouse Quick Start Guide - - - - - - - - - - -
- -
- - -

ClickHouse

-

Tutorial

-
- -

Let's get started with sample dataset from open sources. We will use USA civil flights data since 1987 till 2015. - It's hard to call this sample a Big Data (contains 166 millions rows, 63 Gb of uncompressed data) but this - allows us to quickly get to work. Dataset is available for download here. - Also you may download it from the original datasource as described here.

- -

Firstly we will deploy ClickHouse to a single server. Below that we will also review the process of deployment to - a cluster with support for sharding and replication.

- -

On Ubuntu and Debian Linux ClickHouse can be installed from packages. - For other Linux distributions you can compile - ClickHouse from sources and then install.

- -

clickhouse-client package contains clickhouse-client application — - 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 - element in config. Path determines the location for data storage. It's not really handy to directly - edit config.xml file considering package updates. Recommended way is to override the config elements in - files of config.d directory. - Also you may want to set up access - rights at the start.

- -

clickhouse-server won't be launched automatically after package installation. It won't be automatically - restarted after updates either. Start the server with: -

sudo service clickhouse-server start
- Default location for server logs is /var/log/clickhouse-server/ - Server is ready to handle client connections once "Ready for connections" message was logged.

- -

Use clickhouse-client to connect to the server.

- -
Tips for clickhouse-client -
- Interactive mode: -
-clickhouse-client
-clickhouse-client --host=... --port=... --user=... --password=...
-
- Enable multiline queries: -
-clickhouse-client -m
-clickhouse-client --multiline
-
- Run queries in batch-mode: -
-clickhouse-client --query='SELECT 1'
-echo 'SELECT 1' | clickhouse-client
-
- Insert data from file of a specified format: -
-clickhouse-client --query='INSERT INTO table VALUES' < data.txt
-clickhouse-client --query='INSERT INTO table FORMAT TabSeparated' < data.tsv
-
-
-
- -

Create table for sample dataset

-
Create table query -
-
-$ clickhouse-client --multiline
-ClickHouse client version 0.0.53720.
-Connecting to localhost:9000.
-Connected to ClickHouse server version 0.0.53720.
-
-:) CREATE TABLE ontime
-(
-    Year UInt16,
-    Quarter UInt8,
-    Month UInt8,
-    DayofMonth UInt8,
-    DayOfWeek UInt8,
-    FlightDate Date,
-    UniqueCarrier FixedString(7),
-    AirlineID Int32,
-    Carrier FixedString(2),
-    TailNum String,
-    FlightNum String,
-    OriginAirportID Int32,
-    OriginAirportSeqID Int32,
-    OriginCityMarketID Int32,
-    Origin FixedString(5),
-    OriginCityName String,
-    OriginState FixedString(2),
-    OriginStateFips String,
-    OriginStateName String,
-    OriginWac Int32,
-    DestAirportID Int32,
-    DestAirportSeqID Int32,
-    DestCityMarketID Int32,
-    Dest FixedString(5),
-    DestCityName String,
-    DestState FixedString(2),
-    DestStateFips String,
-    DestStateName String,
-    DestWac Int32,
-    CRSDepTime Int32,
-    DepTime Int32,
-    DepDelay Int32,
-    DepDelayMinutes Int32,
-    DepDel15 Int32,
-    DepartureDelayGroups String,
-    DepTimeBlk String,
-    TaxiOut Int32,
-    WheelsOff Int32,
-    WheelsOn Int32,
-    TaxiIn Int32,
-    CRSArrTime Int32,
-    ArrTime Int32,
-    ArrDelay Int32,
-    ArrDelayMinutes Int32,
-    ArrDel15 Int32,
-    ArrivalDelayGroups Int32,
-    ArrTimeBlk String,
-    Cancelled UInt8,
-    CancellationCode FixedString(1),
-    Diverted UInt8,
-    CRSElapsedTime Int32,
-    ActualElapsedTime Int32,
-    AirTime Int32,
-    Flights Int32,
-    Distance Int32,
-    DistanceGroup UInt8,
-    CarrierDelay Int32,
-    WeatherDelay Int32,
-    NASDelay Int32,
-    SecurityDelay Int32,
-    LateAircraftDelay Int32,
-    FirstDepTime String,
-    TotalAddGTime String,
-    LongestAddGTime String,
-    DivAirportLandings String,
-    DivReachedDest String,
-    DivActualElapsedTime String,
-    DivArrDelay String,
-    DivDistance String,
-    Div1Airport String,
-    Div1AirportID Int32,
-    Div1AirportSeqID Int32,
-    Div1WheelsOn String,
-    Div1TotalGTime String,
-    Div1LongestGTime String,
-    Div1WheelsOff String,
-    Div1TailNum String,
-    Div2Airport String,
-    Div2AirportID Int32,
-    Div2AirportSeqID Int32,
-    Div2WheelsOn String,
-    Div2TotalGTime String,
-    Div2LongestGTime String,
-    Div2WheelsOff String,
-    Div2TailNum String,
-    Div3Airport String,
-    Div3AirportID Int32,
-    Div3AirportSeqID Int32,
-    Div3WheelsOn String,
-    Div3TotalGTime String,
-    Div3LongestGTime String,
-    Div3WheelsOff String,
-    Div3TailNum String,
-    Div4Airport String,
-    Div4AirportID Int32,
-    Div4AirportSeqID Int32,
-    Div4WheelsOn String,
-    Div4TotalGTime String,
-    Div4LongestGTime String,
-    Div4WheelsOff String,
-    Div4TailNum String,
-    Div5Airport String,
-    Div5AirportID Int32,
-    Div5AirportSeqID Int32,
-    Div5WheelsOn String,
-    Div5TotalGTime String,
-    Div5LongestGTime String,
-    Div5WheelsOff String,
-    Div5TailNum String
-)
-ENGINE = MergeTree(FlightDate, (Year, FlightDate), 8192);
-
-
-
- -

Now we have a table of MergeTree type. - MergeTree table type is recommended for usage in production. Table of this kind has a primary key used for - incremental sort of table data. This allows fast execution of queries in ranges of a primary key.

- - -

Note - We store ad network banners impressions logs in ClickHouse. Each table entry looks like: - [Advertiser ID, Impression ID, attribute1, attribute2, …]. - Let assume that our aim is to provide a set of reports for each advertiser. Common and frequently demanded query - would be to count impressions for a specific Advertiser ID. This means that table primary key should start with - Advertiser ID. In this case ClickHouse needs to read smaller amount of data to perform the query for a - given Advertiser ID. -

- -

Load data

-
xz -v -c -d < ontime.csv.xz | clickhouse-client --query="INSERT INTO ontime FORMAT CSV"
-

ClickHouse INSERT query allows to load data in any supported - format. Data load requires just O(1) RAM consumption. INSERT query can receive any data volume as input. - It's strongly recommended to insert data with not too small - size blocks. Notice that insert of blocks with size up to max_insert_block_size (= 1 048 576 - rows by default) is an atomic operation: data block will be inserted completely or not inserted at all. In case - of disconnect during insert operation you may not know if the block was inserted successfully. To achieve - exactly-once semantics ClickHouse supports idempotency for replicated tables. This means - that you may retry insert of the same data block (possibly on a different replicas) but this block will be - inserted just once. Anyway in this guide we will load data from our localhost so we may not take care about data - blocks generation and exactly-once semantics.

- -

INSERT query into tables of MergeTree type is non-blocking (so does a SELECT query). You can execute SELECT - queries right after of during insert operation.

- -

Our sample dataset is a bit not optimal. There are two reasons.

- -

The first is that String data type is used in cases when Enum or numeric type would fit best.

- -

When set of possible values is determined and known to be small. (E.g. OS name, browser - vendors etc.) it's recommended to use Enums or numbers to improve performance. - When set of possible values is not limited (search query, URL, etc.) just go ahead with String.

- -

The second is that dataset contains redundant fields like Year, Quarter, Month, DayOfMonth, DayOfWeek. In fact a - single FlightDate would be enough. Most likely they have been added to improve performance for other DBMS'es - which DateTime handling functions may be not efficient.

- -

ClickHouse functions - for operating with DateTime fields are well-optimized so such redundancy is not required. Anyway much - columns is not a reason to worry — ClickHouse is a column-oriented - DBMS. This allows you to have as much fields as you need. Hundreds of columns in a table is fine for - ClickHouse.

- -

Querying the sample dataset

- -

Here are some examples of the queries from our test data.

- -
    -
  • -
    the most popular destinations in 2015; -
    -
    -SELECT
    -    OriginCityName,
    -    DestCityName,
    -    count(*) AS flights,
    -    bar(flights, 0, 20000, 40)
    -FROM ontime WHERE Year = 2015 GROUP BY OriginCityName, DestCityName ORDER BY flights DESC LIMIT 20
    -
    - -
    -SELECT
    -    OriginCityName < DestCityName ? OriginCityName : DestCityName AS a,
    -    OriginCityName < DestCityName ? DestCityName : OriginCityName AS b,
    -    count(*) AS flights,
    -    bar(flights, 0, 40000, 40)
    -FROM ontime WHERE Year = 2015 GROUP BY a, b ORDER BY flights DESC LIMIT 20
    -
    -
    -
    -
  • -
  • -
    the most popular cities of departure; -
    -
    -SELECT OriginCityName, count(*) AS flights
    -FROM ontime GROUP BY OriginCityName ORDER BY flights DESC LIMIT 20
    -
    -
    -
    -
  • -
  • -
    cities of departure which offer maximum variety of - destinations; -
    -
    -SELECT OriginCityName, uniq(Dest) AS u
    -FROM ontime GROUP BY OriginCityName ORDER BY u DESC LIMIT 20
    -
    -
    -
    -
  • -
  • -
    flight delay dependence on the day of week; -
    -
    -SELECT DayOfWeek, count() AS c, avg(DepDelay >  60) AS delays
    -FROM ontime GROUP BY DayOfWeek ORDER BY DayOfWeek
    -
    -
    -
    -
  • -
  • -
    cities of departure with most frequent delays for 1 hour or - longer; -
    -
    -SELECT OriginCityName, count() AS c, avg(DepDelay >  60) AS delays
    -FROM ontime
    -GROUP BY OriginCityName
    -HAVING c >  100000
    -ORDER BY delays DESC
    -LIMIT 20
    -
    -
    -
    -
  • -
  • -
    flights of maximum duration; -
    -
    -SELECT OriginCityName, DestCityName, count(*) AS flights, avg(AirTime) AS duration
    -FROM ontime
    -GROUP BY OriginCityName, DestCityName
    -ORDER BY duration DESC
    -LIMIT 20
    -
    -
    -
    -
  • -
  • -
    distribution of arrival time delays split by aircompanies; -
    -
    -SELECT Carrier, count() AS c, round(quantileTDigest(0.99)(DepDelay), 2) AS q
    -FROM ontime GROUP BY Carrier ORDER BY q DESC
    -
    -
    -
    -
  • -
  • -
    aircompanies who stopped flights operation; -
    -
    -SELECT Carrier, min(Year), max(Year), count()
    -FROM ontime GROUP BY Carrier HAVING max(Year) < 2015 ORDER BY count() DESC
    -
    -
    -
    -
  • -
  • -
    most trending destination cities in 2015; -
    -
    -SELECT
    -    DestCityName,
    -    sum(Year = 2014) AS c2014,
    -    sum(Year = 2015) AS c2015,
    -    c2015 / c2014 AS diff
    -FROM ontime
    -WHERE Year IN (2014, 2015)
    -GROUP BY DestCityName
    -HAVING c2014 >  10000 AND c2015 >  1000 AND diff >  1
    -ORDER BY diff DESC
    -
    -
    -
    -
  • -
  • -
    destination cities with maximum popularity-season - dependency. -
    -
    -SELECT
    -    DestCityName,
    -    any(total),
    -    avg(abs(monthly * 12 - total) / total) AS avg_month_diff
    -FROM
    -(
    -    SELECT DestCityName, count() AS total
    -    FROM ontime GROUP BY DestCityName HAVING total > 100000
    -)
    -ALL INNER JOIN
    -(
    -    SELECT DestCityName, Month, count() AS monthly
    -    FROM ontime GROUP BY DestCityName, Month HAVING monthly > 10000
    -)
    -USING DestCityName
    -GROUP BY DestCityName
    -ORDER BY avg_month_diff DESC
    -LIMIT 20
    -
    -
    -
    -
  • -
- -

ClickHouse deployment to cluster

-

ClickHouse cluster is a homogenous cluster. Steps to set up: -

    -
  1. Install ClickHouse server on all machines of the cluster
  2. -
  3. Set up cluster configs in configuration file
  4. -
  5. Create local tables on each instance
  6. -
  7. Create a Distributed table
  8. -
-

- -

Distributed-table is actually a kind of - "view" to local tables of ClickHouse cluster. SELECT query from a distributed table will be executed using - resources of all cluster's shards. You may specify configs for multiple clusters and create multiple - Distributed-tables providing views to different clusters.

- -
Config for cluster of three shards. Each shard stores data on a single - replica -
-
-<remote_servers>
-    <perftest_3shards_1replicas>
-        <shard>
-            <replica>
-                <host>example-perftest01j.yandex.ru</host>
-                <port>9000</port>
-            </replica>
-        </shard>
-        <shard>
-            <replica>
-                <host>example-perftest02j.yandex.ru</host>
-                <port>9000</port>
-            </replica>
-        </shard>
-        <shard>
-            <replica>
-                <host>example-perftest03j.yandex.ru</host>
-                <port>9000</port>
-            </replica>
-        </shard>
-    </perftest_3shards_1replicas>
-</remote_servers>
-
-
-
- Creating a local table: -
CREATE TABLE ontime_local (...) ENGINE = MergeTree(FlightDate, (Year, FlightDate), 8192);
- Creating a distributed table providing a view into local tables of the cluster: -
CREATE TABLE ontime_all AS ontime_local
-    ENGINE = Distributed(perftest_3shards_1replicas, default, ontime_local, rand());
- -

You can create a Distributed table on all machines in the cluster. This would allow to run distributed queries on - any machine of the cluster. Besides distributed table you can also use *remote* table function.

- -

Let's run INSERT SELECT into Distributed table - to spread the table to multiple servers.

- -
INSERT INTO ontime_all SELECT * FROM ontime;
- -

Worth to notice that the approach given above wouldn't fit for sharding of large - tables.

- -

As you could expect heavy queries are executed N times faster being launched on 3 servers instead of one.

-
See here -
- - -

You may have noticed that quantiles calculation are slightly different. This happens due to t-digest - algorithm implementation which is non-deterministic — it depends on the order of data processing.

-
-
- -

In this case we have used a cluster with 3 shards each contains a single replica.

- -

To provide for resilience in production environment we recommend that each shard should contain 2-3 replicas - distributed between multiple data-centers. Note that ClickHouse supports unlimited number of replicas.

- -
Config for cluster of one shard containing three replicas -
-
-<remote_servers>
-    ...
-    <perftest_1shards_3replicas>
-        <shard>
-            <replica>
-                <host>example-perftest01j.yandex.ru</host>
-                <port>9000</port>
-             </replica>
-             <replica>
-                <host>example-perftest02j.yandex.ru</host>
-                <port>9000</port>
-             </replica>
-             <replica>
-                <host>example-perftest03j.yandex.ru</host>
-                <port>9000</port>
-             </replica>
-        </shard>
-    </perftest_1shards_3replicas>
-</remote_servers>
-
-
-
- -

To enable replication ZooKeeper is required. - ClickHouse will take care of data consistency on all replicas and run restore procedure after failure - automatically. It's recommended to deploy ZooKeeper cluster to separate servers.

- -

ZooKeeper is not a requirement — in some simple cases you can duplicate the data by writing it into all the - replicas from your application code. This approach is not recommended — in this case ClickHouse is not able to - guarantee data consistency on all replicas. This remains the responsibility of your application.

- -
Set ZooKeeper locations in configuration file -
-
-<zookeeper>
-    <node>
-        <host>zoo01.yandex.ru</host>
-        <port>2181</port>
-    </node>
-    <node>
-        <host>zoo02.yandex.ru</host>
-        <port>2181</port>
-    </node>
-    <node>
-        <host>zoo03.yandex.ru</host>
-        <port>2181</port>
-    </node>
-</zookeeper>
-
-
-
- -

Also we need to set macros for identifying shard and replica — it will be used on table creation

-
-<macros>
-    <shard>01</shard>
-    <replica>01</replica>
-</macros>
-
-

If there are no replicas at the moment on replicated table creation — a new first replica will be instantiated. - If there are already live replicas — new replica will clone the data from existing ones. You have an option to - create all replicated tables first and that insert data to it. Another option is to create some replicas and add - the others after or during data insertion.

- -
-CREATE TABLE ontime_replica (...)
-ENGINE = ReplicatedMergeTree(
-    '/clickhouse_perftest/tables/{shard}/ontime',
-    '{replica}',
-    FlightDate,
-    (Year, FlightDate),
-    8192);
-
-

Here we use ReplicatedMergeTree - table type. In parameters we specify ZooKeeper path containing shard and replica identifiers.

- -
INSERT INTO ontime_replica SELECT * FROM ontime;
-

Replication operates in multi-master mode. Data can be loaded into any replica — it will be synced with other - instances automatically. Replication is asynchronous so at a given moment of time not all replicas may contain - recently inserted data. To allow data insertion at least one replica should be up. Others will sync up data and - repair consistency once they will become active again. Please notice that such scheme allows for the possibility - of just appended data loss.

- -

- ClickHouse source code is published under Apache 2.0 License. Software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - KIND, either express or implied.

- - - -
- - - - - - - - - - - From a7a6a197437f274da797a4ed59c8c9d2dc1d4f6f Mon Sep 17 00:00:00 2001 From: Sergei Shtykov Date: Thu, 5 Dec 2019 20:22:41 +0300 Subject: [PATCH 62/77] CLICKHOUSEDOCS-466: links fix. --- docs/fa/query_language/functions/introspection.md | 1 + docs/ja/query_language/functions/introspection.md | 1 + docs/toc_fa.yml | 1 + docs/toc_ja.yml | 1 + docs/toc_zh.yml | 1 + docs/zh/query_language/functions/higher_order_functions.md | 2 +- docs/zh/query_language/functions/introspection.md | 1 + 7 files changed, 7 insertions(+), 1 deletion(-) create mode 120000 docs/fa/query_language/functions/introspection.md create mode 120000 docs/ja/query_language/functions/introspection.md create mode 120000 docs/zh/query_language/functions/introspection.md diff --git a/docs/fa/query_language/functions/introspection.md b/docs/fa/query_language/functions/introspection.md new file mode 120000 index 00000000000..b1a487e9c77 --- /dev/null +++ b/docs/fa/query_language/functions/introspection.md @@ -0,0 +1 @@ +../../../en/query_language/functions/introspection.md \ No newline at end of file diff --git a/docs/ja/query_language/functions/introspection.md b/docs/ja/query_language/functions/introspection.md new file mode 120000 index 00000000000..b1a487e9c77 --- /dev/null +++ b/docs/ja/query_language/functions/introspection.md @@ -0,0 +1 @@ +../../../en/query_language/functions/introspection.md \ No newline at end of file diff --git a/docs/toc_fa.yml b/docs/toc_fa.yml index 30de03f320a..c5a2a7fd80b 100644 --- a/docs/toc_fa.yml +++ b/docs/toc_fa.yml @@ -149,6 +149,7 @@ nav: - 'Working with geographical coordinates': 'query_language/functions/geo.md' - 'Working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' - 'Machine Learning Functions': 'query_language/functions/machine_learning_functions.md' + - 'Introspection': 'query_language/functions/introspection.md' - 'Other': 'query_language/functions/other_functions.md' - 'Aggregate Functions': - 'Introduction': 'query_language/agg_functions/index.md' diff --git a/docs/toc_ja.yml b/docs/toc_ja.yml index d96963953fe..8a2b32b240a 100644 --- a/docs/toc_ja.yml +++ b/docs/toc_ja.yml @@ -121,6 +121,7 @@ nav: - 'Working with geographical coordinates': 'query_language/functions/geo.md' - 'Working with Nullable arguments': 'query_language/functions/functions_for_nulls.md' - 'Machine Learning Functions': 'query_language/functions/machine_learning_functions.md' + - 'Introspection': 'query_language/functions/introspection.md' - 'Other': 'query_language/functions/other_functions.md' - 'Aggregate Functions': - 'Introduction': 'query_language/agg_functions/index.md' diff --git a/docs/toc_zh.yml b/docs/toc_zh.yml index 3f19fe3f72b..7395dcfe145 100644 --- a/docs/toc_zh.yml +++ b/docs/toc_zh.yml @@ -148,6 +148,7 @@ nav: - 'GEO函数': 'query_language/functions/geo.md' - 'Nullable处理函数': 'query_language/functions/functions_for_nulls.md' - '机器学习函数': 'query_language/functions/machine_learning_functions.md' + - 'Introspection': 'query_language/functions/introspection.md' - '其他函数': 'query_language/functions/other_functions.md' - '聚合函数': - '介绍': 'query_language/agg_functions/index.md' diff --git a/docs/zh/query_language/functions/higher_order_functions.md b/docs/zh/query_language/functions/higher_order_functions.md index e64db0bc8d3..39c6770e5b8 100644 --- a/docs/zh/query_language/functions/higher_order_functions.md +++ b/docs/zh/query_language/functions/higher_order_functions.md @@ -12,7 +12,7 @@ 除了'arrayMap'和'arrayFilter'以外的所有其他函数,都可以省略第一个参数(lambda函数)。在这种情况下,默认返回数组元素本身。 -### arrayMap(func, arr1, ...) +### arrayMap(func, arr1, ...) {#higher_order_functions-array-map} 将arr 将从'func'函数的原始应用程序获得的数组返回到'arr'数组中的每个元素。 diff --git a/docs/zh/query_language/functions/introspection.md b/docs/zh/query_language/functions/introspection.md new file mode 120000 index 00000000000..b1a487e9c77 --- /dev/null +++ b/docs/zh/query_language/functions/introspection.md @@ -0,0 +1 @@ +../../../en/query_language/functions/introspection.md \ No newline at end of file From 44f3ab5f966c01d83cf8ccbc29edb3d03aa31453 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Nov 2019 18:31:56 +0300 Subject: [PATCH 63/77] Extend typeid_cast() for shared_ptr. --- dbms/src/Common/typeid_cast.h | 26 ++++++++++++++++--- .../libcommon/include/ext/shared_ptr_helper.h | 16 ++++++++++++ 2 files changed, 39 insertions(+), 3 deletions(-) diff --git a/dbms/src/Common/typeid_cast.h b/dbms/src/Common/typeid_cast.h index 9285355e788..29ad2e520c0 100644 --- a/dbms/src/Common/typeid_cast.h +++ b/dbms/src/Common/typeid_cast.h @@ -3,8 +3,10 @@ #include #include #include +#include #include +#include #include #include @@ -27,7 +29,7 @@ std::enable_if_t, To> typeid_cast(From & from) { try { - if (typeid(from) == typeid(To)) + if ((typeid(From) == typeid(To)) || (typeid(from) == typeid(To))) return static_cast(from); } catch (const std::exception & e) @@ -39,12 +41,13 @@ std::enable_if_t, To> typeid_cast(From & from) DB::ErrorCodes::BAD_CAST); } + template -To typeid_cast(From * from) +std::enable_if_t, To> typeid_cast(From * from) { try { - if (typeid(*from) == typeid(std::remove_pointer_t)) + if ((typeid(From) == typeid(std::remove_pointer_t)) || (typeid(*from) == typeid(std::remove_pointer_t))) return static_cast(from); else return nullptr; @@ -54,3 +57,20 @@ To typeid_cast(From * from) throw DB::Exception(e.what(), DB::ErrorCodes::BAD_CAST); } } + + +template +std::enable_if_t, To> typeid_cast(const std::shared_ptr & from) +{ + try + { + if ((typeid(From) == typeid(typename To::element_type)) || (typeid(*from) == typeid(typename To::element_type))) + return std::static_pointer_cast(from); + else + return nullptr; + } + catch (const std::exception & e) + { + throw DB::Exception(e.what(), DB::ErrorCodes::BAD_CAST); + } +} diff --git a/libs/libcommon/include/ext/shared_ptr_helper.h b/libs/libcommon/include/ext/shared_ptr_helper.h index ca7219e6261..df132382fa6 100644 --- a/libs/libcommon/include/ext/shared_ptr_helper.h +++ b/libs/libcommon/include/ext/shared_ptr_helper.h @@ -20,4 +20,20 @@ struct shared_ptr_helper } }; + +template +struct is_shared_ptr +{ + static constexpr bool value = false; +}; + + +template +struct is_shared_ptr> +{ + static constexpr bool value = true; +}; + +template +inline constexpr bool is_shared_ptr_v = is_shared_ptr::value; } From 7db6c60bb11d2f4bf74f5d8f1d9e2c67de7d22ec Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Nov 2019 18:33:07 +0300 Subject: [PATCH 64/77] Add AccessControlManager and an interface for the storage. --- dbms/src/Access/AccessControlManager.cpp | 38 ++ dbms/src/Access/AccessControlManager.h | 27 ++ dbms/src/Access/IAccessEntity.cpp | 22 + dbms/src/Access/IAccessEntity.h | 49 ++ dbms/src/Access/IAccessStorage.cpp | 450 +++++++++++++++++++ dbms/src/Access/IAccessStorage.h | 209 +++++++++ dbms/src/Access/MemoryAccessStorage.cpp | 358 +++++++++++++++ dbms/src/Access/MemoryAccessStorage.h | 65 +++ dbms/src/Access/MultipleAccessStorage.cpp | 246 ++++++++++ dbms/src/Access/MultipleAccessStorage.h | 53 +++ dbms/src/Access/UsersConfigAccessStorage.cpp | 126 ++++++ dbms/src/Access/UsersConfigAccessStorage.h | 42 ++ dbms/src/Common/ErrorCodes.cpp | 4 + dbms/src/Interpreters/Context.cpp | 17 +- dbms/src/Interpreters/Context.h | 5 +- 15 files changed, 1709 insertions(+), 2 deletions(-) create mode 100644 dbms/src/Access/AccessControlManager.cpp create mode 100644 dbms/src/Access/AccessControlManager.h create mode 100644 dbms/src/Access/IAccessEntity.cpp create mode 100644 dbms/src/Access/IAccessEntity.h create mode 100644 dbms/src/Access/IAccessStorage.cpp create mode 100644 dbms/src/Access/IAccessStorage.h create mode 100644 dbms/src/Access/MemoryAccessStorage.cpp create mode 100644 dbms/src/Access/MemoryAccessStorage.h create mode 100644 dbms/src/Access/MultipleAccessStorage.cpp create mode 100644 dbms/src/Access/MultipleAccessStorage.h create mode 100644 dbms/src/Access/UsersConfigAccessStorage.cpp create mode 100644 dbms/src/Access/UsersConfigAccessStorage.h diff --git a/dbms/src/Access/AccessControlManager.cpp b/dbms/src/Access/AccessControlManager.cpp new file mode 100644 index 00000000000..c1f4d1cbec6 --- /dev/null +++ b/dbms/src/Access/AccessControlManager.cpp @@ -0,0 +1,38 @@ +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + std::vector> createStorages() + { + std::vector> list; + list.emplace_back(std::make_unique()); + list.emplace_back(std::make_unique()); + return list; + } +} + + +AccessControlManager::AccessControlManager() + : MultipleAccessStorage(createStorages()) +{ +} + + +AccessControlManager::~AccessControlManager() +{ +} + + +void AccessControlManager::loadFromConfig(const Poco::Util::AbstractConfiguration & users_config) +{ + auto & users_config_access_storage = dynamic_cast(getStorageByIndex(1)); + users_config_access_storage.loadFromConfig(users_config); +} + +} diff --git a/dbms/src/Access/AccessControlManager.h b/dbms/src/Access/AccessControlManager.h new file mode 100644 index 00000000000..df83b46aae2 --- /dev/null +++ b/dbms/src/Access/AccessControlManager.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + + +namespace Poco +{ + namespace Util + { + class AbstractConfiguration; + } +} + +namespace DB +{ +/// Manages access control entities. +class AccessControlManager : public MultipleAccessStorage +{ +public: + AccessControlManager(); + ~AccessControlManager(); + + void loadFromConfig(const Poco::Util::AbstractConfiguration & users_config); +}; + +} diff --git a/dbms/src/Access/IAccessEntity.cpp b/dbms/src/Access/IAccessEntity.cpp new file mode 100644 index 00000000000..ed12f36a027 --- /dev/null +++ b/dbms/src/Access/IAccessEntity.cpp @@ -0,0 +1,22 @@ +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_NOT_FOUND; +} + + +String IAccessEntity::getTypeName(std::type_index type) +{ + return demangle(type.name()); +} + +bool IAccessEntity::equal(const IAccessEntity & other) const +{ + return (full_name == other.full_name) && (getType() == other.getType()); +} +} diff --git a/dbms/src/Access/IAccessEntity.h b/dbms/src/Access/IAccessEntity.h new file mode 100644 index 00000000000..272fde006ac --- /dev/null +++ b/dbms/src/Access/IAccessEntity.h @@ -0,0 +1,49 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ +/// Access entity is a set of data which have a name and a type. Access entity control something related to the access control. +/// Entities can be stored to a file or another storage, see IAccessStorage. +struct IAccessEntity +{ + IAccessEntity() = default; + IAccessEntity(const IAccessEntity &) = default; + virtual ~IAccessEntity() = default; + virtual std::shared_ptr clone() const = 0; + + std::type_index getType() const { return typeid(*this); } + static String getTypeName(std::type_index type); + const String getTypeName() const { return getTypeName(getType()); } + + template + bool isTypeOf() const { return isTypeOf(typeid(EntityType)); } + bool isTypeOf(std::type_index type) const { return type == getType(); } + + virtual void setName(const String & name_) { full_name = name_; } + virtual String getName() const { return full_name; } + String getFullName() const { return full_name; } + + friend bool operator ==(const IAccessEntity & lhs, const IAccessEntity & rhs) { return lhs.equal(rhs); } + friend bool operator !=(const IAccessEntity & lhs, const IAccessEntity & rhs) { return !(lhs == rhs); } + +protected: + String full_name; + + virtual bool equal(const IAccessEntity & other) const; + + /// Helper function to define clone() in the derived classes. + template + std::shared_ptr cloneImpl() const + { + return std::make_shared(typeid_cast(*this)); + } +}; + +using AccessEntityPtr = std::shared_ptr; +} diff --git a/dbms/src/Access/IAccessStorage.cpp b/dbms/src/Access/IAccessStorage.cpp new file mode 100644 index 00000000000..4283ec9e6dc --- /dev/null +++ b/dbms/src/Access/IAccessStorage.cpp @@ -0,0 +1,450 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int BAD_CAST; + extern const int ACCESS_ENTITY_NOT_FOUND; + extern const int ACCESS_ENTITY_ALREADY_EXISTS; + extern const int ACCESS_ENTITY_FOUND_DUPLICATES; + extern const int ACCESS_ENTITY_STORAGE_READONLY; +} + + +std::vector IAccessStorage::findAll(std::type_index type) const +{ + return findAllImpl(type); +} + + +std::optional IAccessStorage::find(std::type_index type, const String & name) const +{ + return findImpl(type, name); +} + + +std::vector IAccessStorage::find(std::type_index type, const Strings & names) const +{ + std::vector ids; + ids.reserve(names.size()); + for (const String & name : names) + { + auto id = findImpl(type, name); + if (id) + ids.push_back(*id); + } + return ids; +} + + +UUID IAccessStorage::getID(std::type_index type, const String & name) const +{ + auto id = findImpl(type, name); + if (id) + return *id; + throwNotFound(type, name); +} + + +std::vector IAccessStorage::getIDs(std::type_index type, const Strings & names) const +{ + std::vector ids; + ids.reserve(names.size()); + for (const String & name : names) + ids.push_back(getID(type, name)); + return ids; +} + + +bool IAccessStorage::exists(const UUID & id) const +{ + return existsImpl(id); +} + + + +AccessEntityPtr IAccessStorage::tryReadBase(const UUID & id) const +{ + try + { + return readImpl(id); + } + catch (Exception &) + { + return nullptr; + } +} + + +String IAccessStorage::readName(const UUID & id) const +{ + return readNameImpl(id); +} + + +std::optional IAccessStorage::tryReadName(const UUID & id) const +{ + try + { + return readNameImpl(id); + } + catch (Exception &) + { + return {}; + } +} + + +UUID IAccessStorage::insert(const AccessEntityPtr & entity) +{ + return insertImpl(entity, false); +} + + +std::vector IAccessStorage::insert(const std::vector & multiple_entities) +{ + std::vector ids; + ids.reserve(multiple_entities.size()); + String error_message; + for (const auto & entity : multiple_entities) + { + try + { + ids.push_back(insertImpl(entity, false)); + } + catch (Exception & e) + { + if (e.code() != ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS) + throw; + error_message += (error_message.empty() ? "" : ". ") + e.message(); + } + } + if (!error_message.empty()) + throw Exception(error_message, ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); + return ids; +} + + +std::optional IAccessStorage::tryInsert(const AccessEntityPtr & entity) +{ + try + { + return insertImpl(entity, false); + } + catch (Exception &) + { + return {}; + } +} + + +std::vector IAccessStorage::tryInsert(const std::vector & multiple_entities) +{ + std::vector ids; + ids.reserve(multiple_entities.size()); + for (const auto & entity : multiple_entities) + { + try + { + ids.push_back(insertImpl(entity, false)); + } + catch (Exception &) + { + } + } + return ids; +} + + +UUID IAccessStorage::insertOrReplace(const AccessEntityPtr & entity) +{ + return insertImpl(entity, true); +} + + +std::vector IAccessStorage::insertOrReplace(const std::vector & multiple_entities) +{ + std::vector ids; + ids.reserve(multiple_entities.size()); + for (const auto & entity : multiple_entities) + ids.push_back(insertImpl(entity, true)); + return ids; +} + + +void IAccessStorage::remove(const UUID & id) +{ + removeImpl(id); +} + + +void IAccessStorage::remove(const std::vector & ids) +{ + String error_message; + for (const auto & id : ids) + { + try + { + removeImpl(id); + } + catch (Exception & e) + { + if (e.code() != ErrorCodes::ACCESS_ENTITY_NOT_FOUND) + throw; + error_message += (error_message.empty() ? "" : ". ") + e.message(); + } + } + if (!error_message.empty()) + throw Exception(error_message, ErrorCodes::ACCESS_ENTITY_NOT_FOUND); +} + + +bool IAccessStorage::tryRemove(const UUID & id) +{ + try + { + removeImpl(id); + return true; + } + catch (Exception &) + { + return false; + } +} + + +std::vector IAccessStorage::tryRemove(const std::vector & ids) +{ + std::vector removed; + removed.reserve(ids.size()); + for (const auto & id : ids) + { + try + { + removeImpl(id); + removed.push_back(id); + } + catch (Exception &) + { + } + } + return removed; +} + + +void IAccessStorage::update(const UUID & id, const UpdateFunc & update_func) +{ + updateImpl(id, update_func); +} + + +void IAccessStorage::update(const std::vector & ids, const UpdateFunc & update_func) +{ + String error_message; + for (const auto & id : ids) + { + try + { + updateImpl(id, update_func); + } + catch (Exception & e) + { + if (e.code() != ErrorCodes::ACCESS_ENTITY_NOT_FOUND) + throw; + error_message += (error_message.empty() ? "" : ". ") + e.message(); + } + } + if (!error_message.empty()) + throw Exception(error_message, ErrorCodes::ACCESS_ENTITY_NOT_FOUND); +} + + +bool IAccessStorage::tryUpdate(const UUID & id, const UpdateFunc & update_func) +{ + try + { + updateImpl(id, update_func); + return true; + } + catch (Exception &) + { + return false; + } +} + + +std::vector IAccessStorage::tryUpdate(const std::vector & ids, const UpdateFunc & update_func) +{ + std::vector updated; + updated.reserve(ids.size()); + for (const auto & id : ids) + { + try + { + updateImpl(id, update_func); + updated.push_back(id); + } + catch (Exception &) + { + } + } + return updated; +} + + +IAccessStorage::SubscriptionPtr IAccessStorage::subscribeForChanges(std::type_index type, const OnChangedHandler & handler) const +{ + return subscribeForChangesImpl(type, handler); +} + + +IAccessStorage::SubscriptionPtr IAccessStorage::subscribeForChanges(const UUID & id, const OnChangedHandler & handler) const +{ + return subscribeForChangesImpl(id, handler); +} + + +IAccessStorage::SubscriptionPtr IAccessStorage::subscribeForChanges(const std::vector & ids, const OnChangedHandler & handler) const +{ + if (ids.empty()) + return nullptr; + if (ids.size() == 1) + return subscribeForChangesImpl(ids[0], handler); + + std::vector subscriptions; + subscriptions.reserve(ids.size()); + for (const auto & id : ids) + { + auto subscription = subscribeForChangesImpl(id, handler); + if (subscription) + subscriptions.push_back(std::move(subscription)); + } + + class SubscriptionImpl : public Subscription + { + public: + SubscriptionImpl(std::vector subscriptions_) + : subscriptions(std::move(subscriptions_)) {} + private: + std::vector subscriptions; + }; + + return std::make_unique(std::move(subscriptions)); +} + + +bool IAccessStorage::hasSubscription(std::type_index type) const +{ + return hasSubscriptionImpl(type); +} + + +bool IAccessStorage::hasSubscription(const UUID & id) const +{ + return hasSubscriptionImpl(id); +} + + +void IAccessStorage::notify(const Notifications & notifications) +{ + for (const auto & [fn, id, new_entity] : notifications) + fn(id, new_entity); +} + + +UUID IAccessStorage::generateRandomID() +{ + static Poco::UUIDGenerator generator; + UUID id; + generator.createRandom().copyTo(reinterpret_cast(&id)); + return id; +} + + +Poco::Logger * IAccessStorage::getLogger() const +{ + Poco::Logger * ptr = log.load(); + if (!ptr) + log.store(ptr = &Poco::Logger::get("Access(" + storage_name + ")"), std::memory_order_relaxed); + return ptr; +} + + +void IAccessStorage::throwNotFound(const UUID & id) const +{ + throw Exception("ID {" + toString(id) + "} not found in " + getStorageName(), ErrorCodes::ACCESS_ENTITY_NOT_FOUND); +} + + +void IAccessStorage::throwNotFound(std::type_index type, const String & name) const +{ + throw Exception( + getTypeName(type) + " " + backQuote(name) + " not found in " + getStorageName(), ErrorCodes::ACCESS_ENTITY_NOT_FOUND); +} + + +void IAccessStorage::throwBadCast(const UUID & id, std::type_index type, const String & name, std::type_index required_type) const +{ + throw Exception( + "ID {" + toString(id) + "}: " + getTypeName(type) + backQuote(name) + " expected to be of type " + getTypeName(required_type), + ErrorCodes::BAD_CAST); +} + + +void IAccessStorage::throwIDCollisionCannotInsert(const UUID & id, std::type_index type, const String & name, std::type_index existing_type, const String & existing_name) const +{ + throw Exception( + getTypeName(type) + " " + backQuote(name) + ": cannot insert because the ID {" + toString(id) + "} is already used by " + + getTypeName(existing_type) + " " + backQuote(existing_name) + " in " + getStorageName(), + ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); +} + + +void IAccessStorage::throwNameCollisionCannotInsert(std::type_index type, const String & name) const +{ + throw Exception( + getTypeName(type) + " " + backQuote(name) + ": cannot insert because " + getTypeName(type) + " " + backQuote(name) + + " already exists in " + getStorageName(), + ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); +} + + +void IAccessStorage::throwNameCollisionCannotRename(std::type_index type, const String & old_name, const String & new_name) const +{ + throw Exception( + getTypeName(type) + " " + backQuote(old_name) + ": cannot rename to " + backQuote(new_name) + " because " + getTypeName(type) + " " + + backQuote(new_name) + " already exists in " + getStorageName(), + ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); +} + + +void IAccessStorage::throwReadonlyCannotInsert(std::type_index type, const String & name) const +{ + throw Exception( + "Cannot insert " + getTypeName(type) + " " + backQuote(name) + " to " + getStorageName() + " because this storage is readonly", + ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY); +} + + +void IAccessStorage::throwReadonlyCannotUpdate(std::type_index type, const String & name) const +{ + throw Exception( + "Cannot update " + getTypeName(type) + " " + backQuote(name) + " in " + getStorageName() + " because this storage is readonly", + ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY); +} + + +void IAccessStorage::throwReadonlyCannotRemove(std::type_index type, const String & name) const +{ + throw Exception( + "Cannot remove " + getTypeName(type) + " " + backQuote(name) + " from " + getStorageName() + " because this storage is readonly", + ErrorCodes::ACCESS_ENTITY_STORAGE_READONLY); +} +} diff --git a/dbms/src/Access/IAccessStorage.h b/dbms/src/Access/IAccessStorage.h new file mode 100644 index 00000000000..b4153bce87d --- /dev/null +++ b/dbms/src/Access/IAccessStorage.h @@ -0,0 +1,209 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + + +namespace Poco { class Logger; } + +namespace DB +{ +/// Contains entities, i.e. instances of classes derived from IAccessEntity. +/// The implementations of this class MUST be thread-safe. +class IAccessStorage +{ +public: + IAccessStorage(const String & storage_name_) : storage_name(storage_name_) {} + virtual ~IAccessStorage() {} + + /// Returns the name of this storage. + const String & getStorageName() const { return storage_name; } + + /// Returns the identifiers of all the entities of a specified type contained in the storage. + std::vector findAll(std::type_index type) const; + + template + std::vector findAll() const { return findAll(typeid(EntityType)); } + + /// Searchs for an entity with specified type and name. Returns std::nullopt if not found. + std::optional find(std::type_index type, const String & name) const; + + template + std::optional find(const String & name) const { return find(typeid(EntityType), name); } + + std::vector find(std::type_index type, const Strings & names) const; + + template + std::vector find(const Strings & names) const { return find(typeid(EntityType), names); } + + /// Searchs for an entity with specified name and type. Throws an exception if not found. + UUID getID(std::type_index type, const String & name) const; + + template + UUID getID(const String & name) const { return getID(typeid(EntityType), name); } + + std::vector getIDs(std::type_index type, const Strings & names) const; + + template + std::vector getIDs(const Strings & names) const { return getIDs(typeid(EntityType), names); } + + /// Returns whether there is an entity with such identifier in the storage. + bool exists(const UUID & id) const; + + /// Reads an entity. Throws an exception if not found. + template + std::shared_ptr read(const UUID & id) const; + + template + std::shared_ptr read(const String & name) const; + + /// Reads an entity. Returns nullptr if not found. + template + std::shared_ptr tryRead(const UUID & id) const; + + template + std::shared_ptr tryRead(const String & name) const; + + /// Reads only name of an entity. + String readName(const UUID & id) const; + std::optional tryReadName(const UUID & id) const; + + /// Inserts an entity to the storage. Returns ID of a new entry in the storage. + /// Throws an exception if the specified name already exists. + UUID insert(const AccessEntityPtr & entity); + std::vector insert(const std::vector & multiple_entities); + + /// Inserts an entity to the storage. Returns ID of a new entry in the storage. + std::optional tryInsert(const AccessEntityPtr & entity); + std::vector tryInsert(const std::vector & multiple_entities); + + /// Inserts an entity to the storage. Return ID of a new entry in the storage. + /// Replaces an existing entry in the storage if the specified name already exists. + UUID insertOrReplace(const AccessEntityPtr & entity); + std::vector insertOrReplace(const std::vector & multiple_entities); + + /// Removes an entity from the storage. Throws an exception if couldn't remove. + void remove(const UUID & id); + void remove(const std::vector & ids); + + /// Removes an entity from the storage. Returns false if couldn't remove. + bool tryRemove(const UUID & id); + + /// Removes multiple entities from the storage. Returns the list of successfully dropped. + std::vector tryRemove(const std::vector & ids); + + using UpdateFunc = std::function; + + /// Updates an entity stored in the storage. Throws an exception if couldn't update. + void update(const UUID & id, const UpdateFunc & update_func); + void update(const std::vector & ids, const UpdateFunc & update_func); + + /// Updates an entity stored in the storage. Returns false if couldn't update. + bool tryUpdate(const UUID & id, const UpdateFunc & update_func); + + /// Updates multiple entities in the storage. Returns the list of successfully updated. + std::vector tryUpdate(const std::vector & ids, const UpdateFunc & update_func); + + class Subscription + { + public: + virtual ~Subscription() {} + }; + + using SubscriptionPtr = std::unique_ptr; + using OnChangedHandler = std::function; + + /// Subscribes for all changes. + /// Can return nullptr if cannot subscribe (identifier not found) or if it doesn't make sense (the storage is read-only). + SubscriptionPtr subscribeForChanges(std::type_index type, const OnChangedHandler & handler) const; + + template + SubscriptionPtr subscribeForChanges(OnChangedHandler handler) const { return subscribeForChanges(typeid(EntityType), handler); } + + /// Subscribes for changes of a specific entry. + /// Can return nullptr if cannot subscribe (identifier not found) or if it doesn't make sense (the storage is read-only). + SubscriptionPtr subscribeForChanges(const UUID & id, const OnChangedHandler & handler) const; + SubscriptionPtr subscribeForChanges(const std::vector & ids, const OnChangedHandler & handler) const; + + bool hasSubscription(std::type_index type) const; + bool hasSubscription(const UUID & id) const; + +protected: + virtual std::optional findImpl(std::type_index type, const String & name) const = 0; + virtual std::vector findAllImpl(std::type_index type) const = 0; + virtual bool existsImpl(const UUID & id) const = 0; + virtual AccessEntityPtr readImpl(const UUID & id) const = 0; + virtual String readNameImpl(const UUID & id) const = 0; + virtual UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) = 0; + virtual void removeImpl(const UUID & id) = 0; + virtual void updateImpl(const UUID & id, const UpdateFunc & update_func) = 0; + virtual SubscriptionPtr subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const = 0; + virtual SubscriptionPtr subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const = 0; + virtual bool hasSubscriptionImpl(const UUID & id) const = 0; + virtual bool hasSubscriptionImpl(std::type_index type) const = 0; + + static UUID generateRandomID(); + Poco::Logger * getLogger() const; + static String getTypeName(std::type_index type) { return IAccessEntity::getTypeName(type); } + [[noreturn]] void throwNotFound(const UUID & id) const; + [[noreturn]] void throwNotFound(std::type_index type, const String & name) const; + [[noreturn]] void throwBadCast(const UUID & id, std::type_index type, const String & name, std::type_index required_type) const; + [[noreturn]] void throwIDCollisionCannotInsert(const UUID & id, std::type_index type, const String & name, std::type_index existing_type, const String & existing_name) const; + [[noreturn]] void throwNameCollisionCannotInsert(std::type_index type, const String & name) const; + [[noreturn]] void throwNameCollisionCannotRename(std::type_index type, const String & old_name, const String & new_name) const; + [[noreturn]] void throwReadonlyCannotInsert(std::type_index type, const String & name) const; + [[noreturn]] void throwReadonlyCannotUpdate(std::type_index type, const String & name) const; + [[noreturn]] void throwReadonlyCannotRemove(std::type_index type, const String & name) const; + + using Notification = std::tuple; + using Notifications = std::vector; + static void notify(const Notifications & notifications); + +private: + AccessEntityPtr tryReadBase(const UUID & id) const; + + const String storage_name; + mutable std::atomic log = nullptr; +}; + + +template +std::shared_ptr IAccessStorage::read(const UUID & id) const +{ + auto entity = readImpl(id); + auto ptr = typeid_cast>(entity); + if (ptr) + return ptr; + throwBadCast(id, entity->getType(), entity->getFullName(), typeid(EntityType)); +} + + +template +std::shared_ptr IAccessStorage::read(const String & name) const +{ + return read(getID(name)); +} + + +template +std::shared_ptr IAccessStorage::tryRead(const UUID & id) const +{ + auto entity = tryReadBase(id); + if (!entity) + return nullptr; + return typeid_cast>(entity); +} + + +template +std::shared_ptr IAccessStorage::tryRead(const String & name) const +{ + auto id = find(name); + return id ? tryRead(*id) : nullptr; +} +} diff --git a/dbms/src/Access/MemoryAccessStorage.cpp b/dbms/src/Access/MemoryAccessStorage.cpp new file mode 100644 index 00000000000..ed42acca1a7 --- /dev/null +++ b/dbms/src/Access/MemoryAccessStorage.cpp @@ -0,0 +1,358 @@ +#include +#include +#include + + +namespace DB +{ +MemoryAccessStorage::MemoryAccessStorage(const String & storage_name_) + : IAccessStorage(storage_name_), shared_ptr_to_this{std::make_shared(this)} +{ +} + + +MemoryAccessStorage::~MemoryAccessStorage() {} + + +std::optional MemoryAccessStorage::findImpl(std::type_index type, const String & name) const +{ + std::lock_guard lock{mutex}; + auto it = names.find({name, type}); + if (it == names.end()) + return {}; + + Entry & entry = *(it->second); + return entry.id; +} + + +std::vector MemoryAccessStorage::findAllImpl(std::type_index type) const +{ + std::lock_guard lock{mutex}; + std::vector result; + result.reserve(entries.size()); + for (const auto & [id, entry] : entries) + if (entry.entity->isTypeOf(type)) + result.emplace_back(id); + return result; +} + + +bool MemoryAccessStorage::existsImpl(const UUID & id) const +{ + std::lock_guard lock{mutex}; + return entries.count(id); +} + + +AccessEntityPtr MemoryAccessStorage::readImpl(const UUID & id) const +{ + std::lock_guard lock{mutex}; + auto it = entries.find(id); + if (it == entries.end()) + throwNotFound(id); + const Entry & entry = it->second; + return entry.entity; +} + + +String MemoryAccessStorage::readNameImpl(const UUID & id) const +{ + return readImpl(id)->getFullName(); +} + + +UUID MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists) +{ + Notifications notifications; + SCOPE_EXIT({ notify(notifications); }); + + UUID id = generateRandomID(); + std::lock_guard lock{mutex}; + insertNoLock(generateRandomID(), new_entity, replace_if_exists, notifications); + return id; +} + + +void MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications) +{ + const String & name = new_entity->getFullName(); + std::type_index type = new_entity->getType(); + + /// Check that we can insert. + auto it = entries.find(id); + if (it != entries.end()) + { + const auto & existing_entry = it->second; + throwIDCollisionCannotInsert(id, type, name, existing_entry.entity->getType(), existing_entry.entity->getFullName()); + } + + auto it2 = names.find({name, type}); + if (it2 != names.end()) + { + const auto & existing_entry = *(it2->second); + if (replace_if_exists) + removeNoLock(existing_entry.id, notifications); + else + throwNameCollisionCannotInsert(type, name); + } + + /// Do insertion. + auto & entry = entries[id]; + entry.id = id; + entry.entity = new_entity; + names[std::pair{name, type}] = &entry; + prepareNotifications(entry, false, notifications); +} + + +void MemoryAccessStorage::removeImpl(const UUID & id) +{ + Notifications notifications; + SCOPE_EXIT({ notify(notifications); }); + + std::lock_guard lock{mutex}; + removeNoLock(id, notifications); +} + + +void MemoryAccessStorage::removeNoLock(const UUID & id, Notifications & notifications) +{ + auto it = entries.find(id); + if (it == entries.end()) + throwNotFound(id); + + Entry & entry = it->second; + const String & name = entry.entity->getFullName(); + std::type_index type = entry.entity->getType(); + + prepareNotifications(entry, true, notifications); + + /// Do removing. + names.erase({name, type}); + entries.erase(it); +} + + +void MemoryAccessStorage::updateImpl(const UUID & id, const UpdateFunc & update_func) +{ + Notifications notifications; + SCOPE_EXIT({ notify(notifications); }); + + std::lock_guard lock{mutex}; + updateNoLock(id, update_func, notifications); +} + + +void MemoryAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & update_func, Notifications & notifications) +{ + auto it = entries.find(id); + if (it == entries.end()) + throwNotFound(id); + + Entry & entry = it->second; + auto old_entity = entry.entity; + auto new_entity = update_func(old_entity); + + if (*new_entity == *old_entity) + return; + + entry.entity = new_entity; + + if (new_entity->getFullName() != old_entity->getFullName()) + { + auto it2 = names.find({new_entity->getFullName(), new_entity->getType()}); + if (it2 != names.end()) + throwNameCollisionCannotRename(old_entity->getType(), old_entity->getFullName(), new_entity->getFullName()); + + names.erase({old_entity->getFullName(), old_entity->getType()}); + names[std::pair{new_entity->getFullName(), new_entity->getType()}] = &entry; + } + + prepareNotifications(entry, false, notifications); +} + + +void MemoryAccessStorage::setAll(const std::vector & all_entities) +{ + std::vector> entities_with_ids; + entities_with_ids.reserve(all_entities.size()); + for (const auto & entity : all_entities) + entities_with_ids.emplace_back(generateRandomID(), entity); + setAll(entities_with_ids); +} + + +void MemoryAccessStorage::setAll(const std::vector> & all_entities) +{ + Notifications notifications; + SCOPE_EXIT({ notify(notifications); }); + + std::lock_guard lock{mutex}; + setAllNoLock(all_entities, notifications); +} + + +void MemoryAccessStorage::setAllNoLock(const std::vector> & all_entities, Notifications & notifications) +{ + /// Get list of the currently used IDs. Later we will remove those of them which are not used anymore. + std::unordered_set not_used_ids; + for (const auto & id_and_entry : entries) + not_used_ids.emplace(id_and_entry.first); + + /// Remove conflicting entities. + for (const auto & [id, entity] : all_entities) + { + auto it = entries.find(id); + if (it != entries.end()) + { + not_used_ids.erase(id); /// ID is used. + Entry & entry = it->second; + if (entry.entity->getType() != entity->getType()) + { + removeNoLock(id, notifications); + continue; + } + } + auto it2 = names.find({entity->getFullName(), entity->getType()}); + if (it2 != names.end()) + { + Entry & entry = *(it2->second); + if (entry.id != id) + removeNoLock(id, notifications); + } + } + + /// Remove entities which are not used anymore. + for (const auto & id : not_used_ids) + removeNoLock(id, notifications); + + /// Insert or update entities. + for (const auto & [id, entity] : all_entities) + { + auto it = entries.find(id); + if (it != entries.end()) + { + if (*(it->second.entity) != *entity) + { + const AccessEntityPtr & changed_entity = entity; + updateNoLock(id, [&changed_entity](const AccessEntityPtr &) { return changed_entity; }, notifications); + } + } + else + insertNoLock(id, entity, false, notifications); + } +} + + +void MemoryAccessStorage::prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const +{ + for (const auto & handler : entry.handlers_by_id) + notifications.push_back({handler, entry.id, remove ? nullptr : entry.entity}); + + auto range = handlers_by_type.equal_range(entry.entity->getType()); + for (auto it = range.first; it != range.second; ++it) + notifications.push_back({it->second, entry.id, remove ? nullptr : entry.entity}); +} + + +IAccessStorage::SubscriptionPtr MemoryAccessStorage::subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const +{ + class SubscriptionImpl : public Subscription + { + public: + SubscriptionImpl( + const MemoryAccessStorage & storage_, + std::type_index type_, + const OnChangedHandler & handler_) + : storage_weak(storage_.shared_ptr_to_this) + { + std::lock_guard lock{storage_.mutex}; + handler_it = storage_.handlers_by_type.emplace(type_, handler_); + } + + ~SubscriptionImpl() override + { + auto storage = storage_weak.lock(); + if (storage) + { + std::lock_guard lock{(*storage)->mutex}; + (*storage)->handlers_by_type.erase(handler_it); + } + } + + private: + std::weak_ptr storage_weak; + std::unordered_multimap::iterator handler_it; + }; + + return std::make_unique(*this, type, handler); +} + + +IAccessStorage::SubscriptionPtr MemoryAccessStorage::subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const +{ + class SubscriptionImpl : public Subscription + { + public: + SubscriptionImpl( + const MemoryAccessStorage & storage_, + const UUID & id_, + const OnChangedHandler & handler_) + : storage_weak(storage_.shared_ptr_to_this), + id(id_) + { + std::lock_guard lock{storage_.mutex}; + auto it = storage_.entries.find(id); + if (it == storage_.entries.end()) + { + storage_weak.reset(); + return; + } + const Entry & entry = it->second; + handler_it = entry.handlers_by_id.insert(entry.handlers_by_id.end(), handler_); + } + + ~SubscriptionImpl() override + { + auto storage = storage_weak.lock(); + if (storage) + { + std::lock_guard lock{(*storage)->mutex}; + auto it = (*storage)->entries.find(id); + if (it != (*storage)->entries.end()) + { + const Entry & entry = it->second; + entry.handlers_by_id.erase(handler_it); + } + } + } + + private: + std::weak_ptr storage_weak; + UUID id; + std::list::iterator handler_it; + }; + + return std::make_unique(*this, id, handler); +} + + +bool MemoryAccessStorage::hasSubscriptionImpl(const UUID & id) const +{ + auto it = entries.find(id); + if (it != entries.end()) + { + const Entry & entry = it->second; + return !entry.handlers_by_id.empty(); + } + return false; +} + + +bool MemoryAccessStorage::hasSubscriptionImpl(std::type_index type) const +{ + auto range = handlers_by_type.equal_range(type); + return range.first != range.second; +} +} diff --git a/dbms/src/Access/MemoryAccessStorage.h b/dbms/src/Access/MemoryAccessStorage.h new file mode 100644 index 00000000000..51c55487ca7 --- /dev/null +++ b/dbms/src/Access/MemoryAccessStorage.h @@ -0,0 +1,65 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ +/// Implementation of IAccessStorage which keeps all data in memory. +class MemoryAccessStorage : public IAccessStorage +{ +public: + MemoryAccessStorage(const String & storage_name_ = "memory"); + ~MemoryAccessStorage() override; + + /// Sets all entities at once. + void setAll(const std::vector & all_entities); + void setAll(const std::vector> & all_entities); + +private: + std::optional findImpl(std::type_index type, const String & name) const override; + std::vector findAllImpl(std::type_index type) const override; + bool existsImpl(const UUID & id) const override; + AccessEntityPtr readImpl(const UUID & id) const override; + String readNameImpl(const UUID & id) const override; + UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override; + void removeImpl(const UUID & id) override; + void updateImpl(const UUID & id, const UpdateFunc & update_func) override; + SubscriptionPtr subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; + SubscriptionPtr subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const override; + bool hasSubscriptionImpl(const UUID & id) const override; + bool hasSubscriptionImpl(std::type_index type) const override; + + struct Entry + { + UUID id; + AccessEntityPtr entity; + mutable std::list handlers_by_id; + }; + + void insertNoLock(const UUID & id, const AccessEntityPtr & entity, bool replace_if_exists, Notifications & notifications); + void removeNoLock(const UUID & id, Notifications & notifications); + void updateNoLock(const UUID & id, const UpdateFunc & update_func, Notifications & notifications); + void setAllNoLock(const std::vector> & all_entities, Notifications & notifications); + void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const; + + using NameTypePair = std::pair; + struct Hash + { + size_t operator()(const NameTypePair & key) const + { + return std::hash{}(key.first) - std::hash{}(key.second); + } + }; + + mutable std::mutex mutex; + std::unordered_map entries; /// We want to search entries both by ID and by the pair of name and type. + std::unordered_map names; /// and by the pair of name and type. + mutable std::unordered_multimap handlers_by_type; + std::shared_ptr shared_ptr_to_this; /// We need weak pointers to `this` to implement subscriptions. +}; +} diff --git a/dbms/src/Access/MultipleAccessStorage.cpp b/dbms/src/Access/MultipleAccessStorage.cpp new file mode 100644 index 00000000000..f3db0b0fbbc --- /dev/null +++ b/dbms/src/Access/MultipleAccessStorage.cpp @@ -0,0 +1,246 @@ +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int ACCESS_ENTITY_NOT_FOUND; + extern const int ACCESS_ENTITY_FOUND_DUPLICATES; +} + + +namespace +{ + template + String joinStorageNames(const std::vector & storages) + { + String result; + for (const auto & storage : storages) + { + if (!result.empty()) + result += ", "; + result += storage->getStorageName(); + } + return result; + } +} + + +MultipleAccessStorage::MultipleAccessStorage( + std::vector> nested_storages_, size_t index_of_nested_storage_for_insertion_) + : IAccessStorage(joinStorageNames(nested_storages_)) + , nested_storages(std::move(nested_storages_)) + , nested_storage_for_insertion(nested_storages[index_of_nested_storage_for_insertion_].get()) + , ids_cache(512 /* cache size */) +{ +} + + +MultipleAccessStorage::~MultipleAccessStorage() +{ +} + + +std::vector MultipleAccessStorage::findMultiple(std::type_index type, const String & name) const +{ + std::vector ids; + for (const auto & nested_storage : nested_storages) + { + auto id = nested_storage->find(type, name); + if (id) + { + std::lock_guard lock{ids_cache_mutex}; + ids_cache.set(*id, std::make_shared(nested_storage.get())); + ids.push_back(*id); + } + } + return ids; +} + + +std::optional MultipleAccessStorage::findImpl(std::type_index type, const String & name) const +{ + auto ids = findMultiple(type, name); + if (ids.empty()) + return {}; + if (ids.size() == 1) + return ids[0]; + + std::vector storages_with_duplicates; + for (const auto & id : ids) + { + auto * storage = findStorage(id); + if (storage) + storages_with_duplicates.push_back(storage); + } + + throw Exception( + "Found " + getTypeName(type) + " " + backQuote(name) + " in " + std::to_string(ids.size()) + + " storages: " + joinStorageNames(storages_with_duplicates), + ErrorCodes::ACCESS_ENTITY_FOUND_DUPLICATES); +} + + +std::vector MultipleAccessStorage::findAllImpl(std::type_index type) const +{ + std::vector all_ids; + for (const auto & nested_storage : nested_storages) + { + auto ids = nested_storage->findAll(type); + all_ids.insert(all_ids.end(), std::make_move_iterator(ids.begin()), std::make_move_iterator(ids.end())); + } + return all_ids; +} + + +bool MultipleAccessStorage::existsImpl(const UUID & id) const +{ + return findStorage(id) != nullptr; +} + + +IAccessStorage * MultipleAccessStorage::findStorage(const UUID & id) +{ + { + std::lock_guard lock{ids_cache_mutex}; + auto from_cache = ids_cache.get(id); + if (from_cache) + { + auto * storage = *from_cache; + if (storage->exists(id)) + return storage; + } + } + + for (const auto & nested_storage : nested_storages) + { + if (nested_storage->exists(id)) + { + std::lock_guard lock{ids_cache_mutex}; + ids_cache.set(id, std::make_shared(nested_storage.get())); + return nested_storage.get(); + } + } + + return nullptr; +} + + +const IAccessStorage * MultipleAccessStorage::findStorage(const UUID & id) const +{ + return const_cast(this)->findStorage(id); +} + + +IAccessStorage & MultipleAccessStorage::getStorage(const UUID & id) +{ + auto * storage = findStorage(id); + if (storage) + return *storage; + throwNotFound(id); +} + + +const IAccessStorage & MultipleAccessStorage::getStorage(const UUID & id) const +{ + return const_cast(this)->getStorage(id); +} + + +AccessEntityPtr MultipleAccessStorage::readImpl(const UUID & id) const +{ + return getStorage(id).read(id); +} + + +String MultipleAccessStorage::readNameImpl(const UUID & id) const +{ + return getStorage(id).readName(id); +} + + +UUID MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) +{ + auto id = replace_if_exists ? nested_storage_for_insertion->insertOrReplace(entity) : nested_storage_for_insertion->insert(entity); + + std::lock_guard lock{ids_cache_mutex}; + ids_cache.set(id, std::make_shared(nested_storage_for_insertion)); + + return id; +} + + +void MultipleAccessStorage::removeImpl(const UUID & id) +{ + getStorage(id).remove(id); +} + + +void MultipleAccessStorage::updateImpl(const UUID & id, const UpdateFunc & update_func) +{ + getStorage(id).update(id, update_func); +} + + +IAccessStorage::SubscriptionPtr MultipleAccessStorage::subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const +{ + auto storage = findStorage(id); + if (!storage) + return nullptr; + return storage->subscribeForChanges(id, handler); +} + + +IAccessStorage::SubscriptionPtr MultipleAccessStorage::subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const +{ + std::vector subscriptions; + for (const auto & nested_storage : nested_storages) + { + auto subscription = nested_storage->subscribeForChanges(type, handler); + if (subscription) + subscriptions.emplace_back(std::move(subscription)); + } + + if (subscriptions.empty()) + return nullptr; + + if (subscriptions.size() == 1) + return std::move(subscriptions[0]); + + class SubscriptionImpl : public Subscription + { + public: + SubscriptionImpl(std::vector subscriptions_) + : subscriptions(std::move(subscriptions_)) {} + private: + std::vector subscriptions; + }; + + return std::make_unique(std::move(subscriptions)); +} + + +bool MultipleAccessStorage::hasSubscriptionImpl(const UUID & id) const +{ + for (const auto & nested_storage : nested_storages) + { + if (nested_storage->hasSubscription(id)) + return true; + } + return false; +} + + +bool MultipleAccessStorage::hasSubscriptionImpl(std::type_index type) const +{ + for (const auto & nested_storage : nested_storages) + { + if (nested_storage->hasSubscription(type)) + return true; + } + return false; +} +} diff --git a/dbms/src/Access/MultipleAccessStorage.h b/dbms/src/Access/MultipleAccessStorage.h new file mode 100644 index 00000000000..42e500a1851 --- /dev/null +++ b/dbms/src/Access/MultipleAccessStorage.h @@ -0,0 +1,53 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ +/// Implementation of IAccessStorage which contains multiple nested storages. +class MultipleAccessStorage : public IAccessStorage +{ +public: + using Storage = IAccessStorage; + + MultipleAccessStorage(std::vector> nested_storages_, size_t index_of_nested_storage_for_insertion_ = 0); + ~MultipleAccessStorage() override; + + std::vector findMultiple(std::type_index type, const String & name) const; + + template + std::vector findMultiple(const String & name) const { return findMultiple(EntityType::TYPE, name); } + + const Storage * findStorage(const UUID & id) const; + Storage * findStorage(const UUID & id); + const Storage & getStorage(const UUID & id) const; + Storage & getStorage(const UUID & id); + + Storage & getStorageByIndex(size_t i) { return *(nested_storages[i]); } + const Storage & getStorageByIndex(size_t i) const { return *(nested_storages[i]); } + +protected: + std::optional findImpl(std::type_index type, const String & name) const override; + std::vector findAllImpl(std::type_index type) const override; + bool existsImpl(const UUID & id) const override; + AccessEntityPtr readImpl(const UUID & id) const override; + String readNameImpl(const UUID &id) const override; + UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override; + void removeImpl(const UUID & id) override; + void updateImpl(const UUID & id, const UpdateFunc & update_func) override; + SubscriptionPtr subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; + SubscriptionPtr subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const override; + bool hasSubscriptionImpl(const UUID & id) const override; + bool hasSubscriptionImpl(std::type_index type) const override; + +private: + std::vector> nested_storages; + IAccessStorage * nested_storage_for_insertion; + mutable LRUCache ids_cache; + mutable std::mutex ids_cache_mutex; +}; + +} diff --git a/dbms/src/Access/UsersConfigAccessStorage.cpp b/dbms/src/Access/UsersConfigAccessStorage.cpp new file mode 100644 index 00000000000..caabb9ff34e --- /dev/null +++ b/dbms/src/Access/UsersConfigAccessStorage.cpp @@ -0,0 +1,126 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ +#if 0 + char getTypeChar(std::type_index type) + { + UNUSED(type); /// TODO + return 0; + } + + + UUID generateID(std::type_index type, const String & name) + { + Poco::MD5Engine md5; + md5.update(name); + char type_storage_chars[] = " USRSXML"; + type_storage_chars[0] = getTypeChar(type); + md5.update(type_storage_chars, strlen(type_storage_chars)); + UUID result; + memcpy(&result, md5.digest().data(), md5.digestLength()); + return result; + } + + + UUID generateID(const IAccessEntity & entity) { return generateID(entity.getType(), entity.getFullName()); } +#endif +} + + +UsersConfigAccessStorage::UsersConfigAccessStorage() : IAccessStorage("users.xml") +{ +} + + +UsersConfigAccessStorage::~UsersConfigAccessStorage() {} + + +void UsersConfigAccessStorage::loadFromConfig(const Poco::Util::AbstractConfiguration & config) +{ + std::vector> all_entities; + UNUSED(config); /// TODO + memory_storage.setAll(all_entities); +} + + +std::optional UsersConfigAccessStorage::findImpl(std::type_index type, const String & name) const +{ + return memory_storage.find(type, name); +} + + +std::vector UsersConfigAccessStorage::findAllImpl(std::type_index type) const +{ + return memory_storage.findAll(type); +} + + +bool UsersConfigAccessStorage::existsImpl(const UUID & id) const +{ + return memory_storage.exists(id); +} + + +AccessEntityPtr UsersConfigAccessStorage::readImpl(const UUID & id) const +{ + return memory_storage.read(id); +} + + +String UsersConfigAccessStorage::readNameImpl(const UUID & id) const +{ + return memory_storage.readName(id); +} + + +UUID UsersConfigAccessStorage::insertImpl(const AccessEntityPtr & entity, bool) +{ + throwReadonlyCannotInsert(entity->getType(), entity->getFullName()); +} + + +void UsersConfigAccessStorage::removeImpl(const UUID & id) +{ + auto entity = read(id); + throwReadonlyCannotRemove(entity->getType(), entity->getFullName()); +} + + +void UsersConfigAccessStorage::updateImpl(const UUID & id, const UpdateFunc &) +{ + auto entity = read(id); + throwReadonlyCannotUpdate(entity->getType(), entity->getFullName()); +} + + +IAccessStorage::SubscriptionPtr UsersConfigAccessStorage::subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const +{ + return memory_storage.subscribeForChanges(id, handler); +} + + +IAccessStorage::SubscriptionPtr UsersConfigAccessStorage::subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const +{ + return memory_storage.subscribeForChanges(type, handler); +} + + +bool UsersConfigAccessStorage::hasSubscriptionImpl(const UUID & id) const +{ + return memory_storage.hasSubscription(id); +} + + +bool UsersConfigAccessStorage::hasSubscriptionImpl(std::type_index type) const +{ + return memory_storage.hasSubscription(type); +} +} diff --git a/dbms/src/Access/UsersConfigAccessStorage.h b/dbms/src/Access/UsersConfigAccessStorage.h new file mode 100644 index 00000000000..9b0bf2ed17c --- /dev/null +++ b/dbms/src/Access/UsersConfigAccessStorage.h @@ -0,0 +1,42 @@ +#pragma once + +#include + + +namespace Poco +{ + namespace Util + { + class AbstractConfiguration; + } +} + + +namespace DB +{ +/// Implementation of IAccessStorage which loads all from users.xml periodically. +class UsersConfigAccessStorage : public IAccessStorage +{ +public: + UsersConfigAccessStorage(); + ~UsersConfigAccessStorage() override; + + void loadFromConfig(const Poco::Util::AbstractConfiguration & config); + +private: + std::optional findImpl(std::type_index type, const String & name) const override; + std::vector findAllImpl(std::type_index type) const override; + bool existsImpl(const UUID & id) const override; + AccessEntityPtr readImpl(const UUID & id) const override; + String readNameImpl(const UUID & id) const override; + UUID insertImpl(const AccessEntityPtr & entity, bool replace_if_exists) override; + void removeImpl(const UUID & id) override; + void updateImpl(const UUID & id, const UpdateFunc & update_func) override; + SubscriptionPtr subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; + SubscriptionPtr subscribeForChangesImpl(std::type_index type, const OnChangedHandler & handler) const override; + bool hasSubscriptionImpl(const UUID & id) const override; + bool hasSubscriptionImpl(std::type_index type) const override; + + MemoryAccessStorage memory_storage; +}; +} diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index b18fef232ad..0258bcdb6b0 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -466,6 +466,10 @@ namespace ErrorCodes extern const int INCORRECT_DICTIONARY_DEFINITION = 489; extern const int CANNOT_FORMAT_DATETIME = 490; extern const int UNACCEPTABLE_URL = 491; + extern const int ACCESS_ENTITY_NOT_FOUND = 492; + extern const int ACCESS_ENTITY_ALREADY_EXISTS = 493; + extern const int ACCESS_ENTITY_FOUND_DUPLICATES = 494; + extern const int ACCESS_ENTITY_STORAGE_READONLY = 495; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 8dc4e57739f..c1a41a72507 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -24,6 +24,8 @@ #include #include #include +#include +#include #include #include #include @@ -37,7 +39,6 @@ #include #include #include -#include #include #include #include @@ -130,6 +131,7 @@ struct ContextShared mutable std::optional external_models_loader; String default_profile_name; /// Default profile name used for default values. String system_profile_name; /// Profile used by system processes + AccessControlManager access_control_manager; std::unique_ptr users_manager; /// Known users. Quotas quotas; /// Known quotas for resource use. mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. @@ -585,10 +587,23 @@ const Poco::Util::AbstractConfiguration & Context::getConfigRef() const return shared->config ? *shared->config : Poco::Util::Application::instance().config(); } +AccessControlManager & Context::getAccessControlManager() +{ + auto lock = getLock(); + return shared->access_control_manager; +} + +const AccessControlManager & Context::getAccessControlManager() const +{ + auto lock = getLock(); + return shared->access_control_manager; +} + void Context::setUsersConfig(const ConfigurationPtr & config) { auto lock = getLock(); shared->users_config = config; + shared->access_control_manager.loadFromConfig(*shared->users_config); shared->users_manager->loadFromConfig(*shared->users_config); shared->quotas.loadFromConfig(*shared->users_config); } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index e94d8125064..eb561fd92a8 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -77,6 +77,7 @@ class ActionLocksManager; using ActionLocksManagerPtr = std::shared_ptr; class ShellCommand; class ICompressionCodec; +class AccessControlManager; class SettingsConstraints; class RemoteHostFilter; @@ -201,6 +202,9 @@ public: void setConfig(const ConfigurationPtr & config); const Poco::Util::AbstractConfiguration & getConfigRef() const; + AccessControlManager & getAccessControlManager(); + const AccessControlManager & getAccessControlManager() const; + /** Take the list of users, quotas and configuration profiles from this config. * The list of users is completely replaced. * The accumulated quota values are not reset if the quota is not deleted. @@ -410,7 +414,6 @@ public: const Settings & getSettingsRef() const { return settings; } Settings & getSettingsRef() { return settings; } - void setProgressCallback(ProgressCallback callback); /// Used in InterpreterSelectQuery to pass it to the IBlockInputStream. ProgressCallback getProgressCallback() const; From c733e5b50bc5f69460bca3b7c178fe2886396503 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 9 Nov 2019 22:14:51 +0300 Subject: [PATCH 65/77] Add functions for writing std::chrono to string. --- dbms/src/Dictionaries/CacheDictionary.inc.h | 4 +-- dbms/src/Interpreters/ExternalLoader.cpp | 5 ++- libs/libcommon/CMakeLists.txt | 1 + libs/libcommon/include/ext/chrono_io.h | 37 +++++++++++++++++++++ 4 files changed, 42 insertions(+), 5 deletions(-) create mode 100644 libs/libcommon/include/ext/chrono_io.h diff --git a/dbms/src/Dictionaries/CacheDictionary.inc.h b/dbms/src/Dictionaries/CacheDictionary.inc.h index c10cde8c4fd..87005ac821f 100644 --- a/dbms/src/Dictionaries/CacheDictionary.inc.h +++ b/dbms/src/Dictionaries/CacheDictionary.inc.h @@ -3,8 +3,8 @@ #include #include #include -#include #include +#include #include #include #include @@ -334,7 +334,7 @@ void CacheDictionary::update( backoff_end_time = now + std::chrono::seconds(calculateDurationWithBackoff(rnd_engine, error_count)); tryLogException(last_exception, log, "Could not update cache dictionary '" + getName() + - "', next update is scheduled at " + DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(backoff_end_time))); + "', next update is scheduled at " + ext::to_string(backoff_end_time)); } } diff --git a/dbms/src/Interpreters/ExternalLoader.cpp b/dbms/src/Interpreters/ExternalLoader.cpp index 10b8a02d660..b81e549a9d2 100644 --- a/dbms/src/Interpreters/ExternalLoader.cpp +++ b/dbms/src/Interpreters/ExternalLoader.cpp @@ -2,13 +2,13 @@ #include #include -#include #include #include #include #include #include #include +#include #include @@ -878,8 +878,7 @@ private: { if (next_update_time == TimePoint::max()) return String(); - return ", next update is scheduled at " - + DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(next_update_time)); + return ", next update is scheduled at " + ext::to_string(next_update_time); }; if (previous_version) tryLogException(new_exception, log, "Could not update " + type_name + " '" + name + "'" diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 357e457b240..3e58cba0164 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -53,6 +53,7 @@ add_library (common include/common/phdr_cache.h include/ext/bit_cast.h + include/ext/chrono_io.h include/ext/collection_cast.h include/ext/enumerate.h include/ext/function_traits.h diff --git a/libs/libcommon/include/ext/chrono_io.h b/libs/libcommon/include/ext/chrono_io.h new file mode 100644 index 00000000000..8fa448b9e6a --- /dev/null +++ b/libs/libcommon/include/ext/chrono_io.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include + + +namespace ext +{ + template + std::string to_string(const std::chrono::time_point & tp) + { + return DateLUT::instance().timeToString(std::chrono::system_clock::to_time_t(tp)); + } + + template > + std::string to_string(const std::chrono::duration & dur) + { + auto seconds_as_int = std::chrono::duration_cast(dur); + if (seconds_as_int == dur) + return std::to_string(seconds_as_int.count()) + "s"; + auto seconds_as_double = std::chrono::duration_cast>(dur); + return std::to_string(seconds_as_double.count()) + "s"; + } + + template + std::ostream & operator<<(std::ostream & o, const std::chrono::time_point & tp) + { + return o << to_string(tp); + } + + template > + std::ostream & operator<<(std::ostream & o, const std::chrono::duration & dur) + { + return o << to_string(dur); + } +} From e3594bec5ff277740b74304c520f78b03154ab5a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 24 Nov 2019 18:03:30 +0300 Subject: [PATCH 66/77] Add function ext::range_with_static_cast(). --- libs/libcommon/include/ext/range.h | 64 ++++++++++++++---------------- 1 file changed, 30 insertions(+), 34 deletions(-) diff --git a/libs/libcommon/include/ext/range.h b/libs/libcommon/include/ext/range.h index 61b644c2ce5..c379d453f7b 100644 --- a/libs/libcommon/include/ext/range.h +++ b/libs/libcommon/include/ext/range.h @@ -1,46 +1,42 @@ #pragma once #include -#include -#include -#include +#include +#include -/** Numeric range iterator, used to represent a half-closed interval [begin, end). - * In conjunction with std::reverse_iterator allows for forward and backward iteration - * over corresponding interval. - */ namespace ext { - template - using range_iterator = boost::counting_iterator; - - /** Range-based for loop adapter for (reverse_)range_iterator. - * By and large should be in conjunction with ext::range and ext::reverse_range. - */ - template - struct range_wrapper + /// For loop adaptor which is used to iterate through a half-closed interval [begin, end). + template + inline auto range(BeginType begin, EndType end) { - using value_type = typename std::remove_reference::type; - using iterator = range_iterator; + using CommonType = typename std::common_type::type; + return boost::counting_range(begin, end); + } - value_type begin_; - value_type end_; - - iterator begin() const { return iterator(begin_); } - iterator end() const { return iterator(end_); } - }; - - /** Constructs range_wrapper for forward-iteration over [begin, end) in range-based for loop. - * Usage example: - * for (const auto i : ext::range(0, 4)) print(i); - * Output: - * 0 1 2 3 - */ - template - inline range_wrapper::type> range(T1 begin, T2 end) + template + inline auto range(Type end) { - using common_type = typename std::common_type::type; - return { static_cast(begin), static_cast(end) }; + return range(static_cast(0), end); + } + + /// The same as range(), but every value is casted statically to a specified `ValueType`. + /// This is useful to iterate through all constants of a enum. + template + inline auto range_with_static_cast(BeginType begin, EndType end) + { + using CommonType = typename std::common_type::type; + if constexpr (std::is_same_v) + return boost::counting_range(begin, end); + else + return boost::counting_range(begin, end) + | boost::adaptors::transformed([](CommonType x) -> ValueType { return static_cast(x); }); + } + + template + inline auto range_with_static_cast(EndType end) + { + return range_with_static_cast(static_cast(0), end); } } From 0e2167327145ff61d713d443e6715540b0cccb26 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 4 Nov 2019 22:17:27 +0300 Subject: [PATCH 67/77] Rework Quota based on IAccessEntity. --- dbms/programs/server/HTTPHandler.cpp | 1 - dbms/programs/server/TCPHandler.cpp | 1 - dbms/src/Access/AccessControlManager.cpp | 16 +- dbms/src/Access/AccessControlManager.h | 18 + dbms/src/Access/IAccessEntity.cpp | 9 +- dbms/src/Access/Quota.cpp | 46 +++ dbms/src/Access/Quota.h | 141 +++++++ dbms/src/Access/QuotaContext.cpp | 264 ++++++++++++++ dbms/src/Access/QuotaContext.h | 110 ++++++ dbms/src/Access/QuotaContextFactory.cpp | 299 +++++++++++++++ dbms/src/Access/QuotaContextFactory.h | 62 ++++ dbms/src/Access/UsersConfigAccessStorage.cpp | 91 ++++- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/DataStreams/IBlockInputStream.cpp | 18 +- dbms/src/DataStreams/IBlockInputStream.h | 10 +- dbms/src/Interpreters/Context.cpp | 28 +- dbms/src/Interpreters/Context.h | 8 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- dbms/src/Interpreters/Quota.cpp | 345 ------------------ dbms/src/Interpreters/Quota.h | 263 ------------- dbms/src/Interpreters/Users.cpp | 1 - dbms/src/Interpreters/Users.h | 1 - dbms/src/Interpreters/executeQuery.cpp | 13 +- .../TreeExecutorBlockInputStream.cpp | 2 +- .../Executors/TreeExecutorBlockInputStream.h | 2 +- dbms/src/Processors/Pipe.cpp | 2 +- dbms/src/Processors/Pipe.h | 4 +- .../Sources/SourceFromInputStream.h | 2 +- .../Processors/Sources/SourceWithProgress.cpp | 8 +- .../Processors/Sources/SourceWithProgress.h | 6 +- .../Transforms/LimitsCheckingTransform.cpp | 12 +- .../Transforms/LimitsCheckingTransform.h | 6 +- 32 files changed, 1092 insertions(+), 700 deletions(-) create mode 100644 dbms/src/Access/Quota.cpp create mode 100644 dbms/src/Access/Quota.h create mode 100644 dbms/src/Access/QuotaContext.cpp create mode 100644 dbms/src/Access/QuotaContext.h create mode 100644 dbms/src/Access/QuotaContextFactory.cpp create mode 100644 dbms/src/Access/QuotaContextFactory.h delete mode 100644 dbms/src/Interpreters/Quota.cpp delete mode 100644 dbms/src/Interpreters/Quota.h diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index cefa3712997..29d186def2d 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -34,7 +34,6 @@ #include #include #include -#include #include #include diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index 76ea69cc737..7103769d54e 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Access/AccessControlManager.cpp b/dbms/src/Access/AccessControlManager.cpp index c1f4d1cbec6..1f1a57816a8 100644 --- a/dbms/src/Access/AccessControlManager.cpp +++ b/dbms/src/Access/AccessControlManager.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -19,7 +20,8 @@ namespace AccessControlManager::AccessControlManager() - : MultipleAccessStorage(createStorages()) + : MultipleAccessStorage(createStorages()), + quota_context_factory(std::make_unique(*this)) { } @@ -35,4 +37,16 @@ void AccessControlManager::loadFromConfig(const Poco::Util::AbstractConfiguratio users_config_access_storage.loadFromConfig(users_config); } + +std::shared_ptr AccessControlManager::createQuotaContext( + const String & user_name, const Poco::Net::IPAddress & address, const String & custom_quota_key) +{ + return quota_context_factory->createContext(user_name, address, custom_quota_key); +} + + +std::vector AccessControlManager::getQuotaUsageInfo() const +{ + return quota_context_factory->getUsageInfo(); +} } diff --git a/dbms/src/Access/AccessControlManager.h b/dbms/src/Access/AccessControlManager.h index df83b46aae2..2133717d676 100644 --- a/dbms/src/Access/AccessControlManager.h +++ b/dbms/src/Access/AccessControlManager.h @@ -2,10 +2,15 @@ #include #include +#include namespace Poco { + namespace Net + { + class IPAddress; + } namespace Util { class AbstractConfiguration; @@ -14,6 +19,11 @@ namespace Poco namespace DB { +class QuotaContext; +class QuotaContextFactory; +struct QuotaUsageInfo; + + /// Manages access control entities. class AccessControlManager : public MultipleAccessStorage { @@ -22,6 +32,14 @@ public: ~AccessControlManager(); void loadFromConfig(const Poco::Util::AbstractConfiguration & users_config); + + std::shared_ptr + createQuotaContext(const String & user_name, const Poco::Net::IPAddress & address, const String & custom_quota_key); + + std::vector getQuotaUsageInfo() const; + +private: + std::unique_ptr quota_context_factory; }; } diff --git a/dbms/src/Access/IAccessEntity.cpp b/dbms/src/Access/IAccessEntity.cpp index ed12f36a027..6a2f928ae9e 100644 --- a/dbms/src/Access/IAccessEntity.cpp +++ b/dbms/src/Access/IAccessEntity.cpp @@ -1,17 +1,14 @@ #include +#include #include namespace DB { -namespace ErrorCodes -{ - extern const int ACCESS_ENTITY_NOT_FOUND; -} - - String IAccessEntity::getTypeName(std::type_index type) { + if (type == typeid(Quota)) + return "Quota"; return demangle(type.name()); } diff --git a/dbms/src/Access/Quota.cpp b/dbms/src/Access/Quota.cpp new file mode 100644 index 00000000000..d178307ca51 --- /dev/null +++ b/dbms/src/Access/Quota.cpp @@ -0,0 +1,46 @@ +#include +#include +#include + + +namespace DB +{ +Quota::Limits::Limits() +{ + boost::range::fill(max, 0); +} + + +bool operator ==(const Quota::Limits & lhs, const Quota::Limits & rhs) +{ + return boost::range::equal(lhs.max, rhs.max) && (lhs.duration == rhs.duration) + && (lhs.randomize_interval == rhs.randomize_interval); +} + + +bool Quota::equal(const IAccessEntity & other) const +{ + if (!IAccessEntity::equal(other)) + return false; + const auto & other_quota = typeid_cast(other); + return (all_limits == other_quota.all_limits) && (key_type == other_quota.key_type) && (roles == other_quota.roles) + && (all_roles == other_quota.all_roles) && (except_roles == other_quota.except_roles); +} + + +const char * Quota::resourceTypeToColumnName(ResourceType resource_type) +{ + switch (resource_type) + { + case Quota::QUERIES: return "queries"; + case Quota::ERRORS: return "errors"; + case Quota::RESULT_ROWS: return "result_rows"; + case Quota::RESULT_BYTES: return "result_bytes"; + case Quota::READ_ROWS: return "read_rows"; + case Quota::READ_BYTES: return "read_bytes"; + case Quota::EXECUTION_TIME: return "execution_time"; + } + __builtin_unreachable(); +} +} + diff --git a/dbms/src/Access/Quota.h b/dbms/src/Access/Quota.h new file mode 100644 index 00000000000..716bccbe1ff --- /dev/null +++ b/dbms/src/Access/Quota.h @@ -0,0 +1,141 @@ +#pragma once + +#include +#include + + + +namespace DB +{ +/** Quota for resources consumption for specific interval. + * Used to limit resource usage by user. + * Quota is applied "softly" - could be slightly exceed, because it is checked usually only on each block of processed data. + * Accumulated values are not persisted and are lost on server restart. + * Quota is local to server, + * but for distributed queries, accumulated values for read rows and bytes + * are collected from all participating servers and accumulated locally. + */ +struct Quota : public IAccessEntity +{ + enum ResourceType + { + QUERIES, /// Number of queries. + ERRORS, /// Number of queries with exceptions. + RESULT_ROWS, /// Number of rows returned as result. + RESULT_BYTES, /// Number of bytes returned as result. + READ_ROWS, /// Number of rows read from tables. + READ_BYTES, /// Number of bytes read from tables. + EXECUTION_TIME, /// Total amount of query execution time in nanoseconds. + }; + static constexpr size_t MAX_RESOURCE_TYPE = 7; + + using ResourceAmount = UInt64; + static constexpr ResourceAmount UNLIMITED = 0; /// 0 means unlimited. + + /// Amount of resources available to consume for each duration. + struct Limits + { + ResourceAmount max[MAX_RESOURCE_TYPE]; + std::chrono::seconds duration = std::chrono::seconds::zero(); + + /// Intervals can be randomized (to avoid DoS if intervals for many users end at one time). + bool randomize_interval = false; + + Limits(); + friend bool operator ==(const Limits & lhs, const Limits & rhs); + friend bool operator !=(const Limits & lhs, const Limits & rhs) { return !(lhs == rhs); } + }; + + std::vector all_limits; + + /// Key to share quota consumption. + /// Users with the same key share the same amount of resource. + enum class KeyType + { + NONE, /// All users share the same quota. + USER_NAME, /// Connections with the same user name share the same quota. + IP_ADDRESS, /// Connections from the same IP share the same quota. + CLIENT_KEY, /// Client should explicitly supply a key to use. + CLIENT_KEY_OR_USER_NAME, /// Same as CLIENT_KEY, but use USER_NAME if the client doesn't supply a key. + CLIENT_KEY_OR_IP_ADDRESS, /// Same as CLIENT_KEY, but use IP_ADDRESS if the client doesn't supply a key. + }; + static constexpr size_t MAX_KEY_TYPE = 6; + KeyType key_type = KeyType::NONE; + + /// Which roles or users should use this quota. + Strings roles; + bool all_roles = false; + Strings except_roles; + + bool equal(const IAccessEntity & other) const override; + std::shared_ptr clone() const override { return cloneImpl(); } + + static const char * getNameOfResourceType(ResourceType resource_type); + static const char * resourceTypeToKeyword(ResourceType resource_type); + static const char * resourceTypeToColumnName(ResourceType resource_type); + static const char * getNameOfKeyType(KeyType key_type); + static double executionTimeToSeconds(ResourceAmount ns); + static ResourceAmount secondsToExecutionTime(double s); +}; + + +inline const char * Quota::getNameOfResourceType(ResourceType resource_type) +{ + switch (resource_type) + { + case Quota::QUERIES: return "queries"; + case Quota::ERRORS: return "errors"; + case Quota::RESULT_ROWS: return "result rows"; + case Quota::RESULT_BYTES: return "result bytes"; + case Quota::READ_ROWS: return "read rows"; + case Quota::READ_BYTES: return "read bytes"; + case Quota::EXECUTION_TIME: return "execution time"; + } + __builtin_unreachable(); +} + + +inline const char * Quota::resourceTypeToKeyword(ResourceType resource_type) +{ + switch (resource_type) + { + case Quota::QUERIES: return "QUERIES"; + case Quota::ERRORS: return "ERRORS"; + case Quota::RESULT_ROWS: return "RESULT ROWS"; + case Quota::RESULT_BYTES: return "RESULT BYTES"; + case Quota::READ_ROWS: return "READ ROWS"; + case Quota::READ_BYTES: return "READ BYTES"; + case Quota::EXECUTION_TIME: return "EXECUTION TIME"; + } + __builtin_unreachable(); +} + + +inline const char * Quota::getNameOfKeyType(KeyType key_type) +{ + switch (key_type) + { + case KeyType::NONE: return "none"; + case KeyType::USER_NAME: return "user name"; + case KeyType::IP_ADDRESS: return "ip address"; + case KeyType::CLIENT_KEY: return "client key"; + case KeyType::CLIENT_KEY_OR_USER_NAME: return "client key or user name"; + case KeyType::CLIENT_KEY_OR_IP_ADDRESS: return "client key or ip address"; + } + __builtin_unreachable(); +} + + +inline double Quota::executionTimeToSeconds(ResourceAmount ns) +{ + return std::chrono::duration_cast>(std::chrono::nanoseconds{ns}).count(); +} + +inline Quota::ResourceAmount Quota::secondsToExecutionTime(double s) +{ + return std::chrono::duration_cast(std::chrono::duration(s)).count(); +} + + +using QuotaPtr = std::shared_ptr; +} diff --git a/dbms/src/Access/QuotaContext.cpp b/dbms/src/Access/QuotaContext.cpp new file mode 100644 index 00000000000..11666e5d4b8 --- /dev/null +++ b/dbms/src/Access/QuotaContext.cpp @@ -0,0 +1,264 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int QUOTA_EXPIRED; +} + +struct QuotaContext::Impl +{ + [[noreturn]] static void throwQuotaExceed( + const String & user_name, + const String & quota_name, + ResourceType resource_type, + ResourceAmount used, + ResourceAmount max, + std::chrono::seconds duration, + std::chrono::system_clock::time_point end_of_interval) + { + std::function amount_to_string = [](UInt64 amount) { return std::to_string(amount); }; + if (resource_type == Quota::EXECUTION_TIME) + amount_to_string = [&](UInt64 amount) { return ext::to_string(std::chrono::nanoseconds(amount)); }; + + throw Exception( + "Quota for user " + backQuote(user_name) + " for " + ext::to_string(duration) + " has been exceeded: " + + Quota::getNameOfResourceType(resource_type) + " = " + amount_to_string(used) + "/" + amount_to_string(max) + ". " + + "Interval will end at " + ext::to_string(end_of_interval) + ". " + "Name of quota template: " + backQuote(quota_name), + ErrorCodes::QUOTA_EXPIRED); + } + + + static std::chrono::system_clock::time_point getEndOfInterval( + const Interval & interval, std::chrono::system_clock::time_point current_time, bool * counters_were_reset = nullptr) + { + auto & end_of_interval = interval.end_of_interval; + auto end_loaded = end_of_interval.load(); + auto end = std::chrono::system_clock::time_point{end_loaded}; + if (current_time < end) + { + if (counters_were_reset) + *counters_were_reset = false; + return end; + } + + const auto duration = interval.duration; + + do + { + end = end + (current_time - end + duration) / duration * duration; + if (end_of_interval.compare_exchange_strong(end_loaded, end.time_since_epoch())) + { + boost::range::fill(interval.used, 0); + break; + } + end = std::chrono::system_clock::time_point{end_loaded}; + } + while (current_time >= end); + + if (counters_were_reset) + *counters_were_reset = true; + return end; + } + + + static void used( + const String & user_name, + const Intervals & intervals, + ResourceType resource_type, + ResourceAmount amount, + std::chrono::system_clock::time_point current_time, + bool check_exceeded) + { + for (const auto & interval : intervals.intervals) + { + ResourceAmount used = (interval.used[resource_type] += amount); + ResourceAmount max = interval.max[resource_type]; + if (max == Quota::UNLIMITED) + continue; + if (used > max) + { + bool counters_were_reset = false; + auto end_of_interval = getEndOfInterval(interval, current_time, &counters_were_reset); + if (counters_were_reset) + { + used = (interval.used[resource_type] += amount); + if ((used > max) && check_exceeded) + throwQuotaExceed(user_name, intervals.quota_name, resource_type, used, max, interval.duration, end_of_interval); + } + else if (check_exceeded) + throwQuotaExceed(user_name, intervals.quota_name, resource_type, used, max, interval.duration, end_of_interval); + } + } + } + + static void checkExceeded( + const String & user_name, + const Intervals & intervals, + ResourceType resource_type, + std::chrono::system_clock::time_point current_time) + { + for (const auto & interval : intervals.intervals) + { + ResourceAmount used = interval.used[resource_type]; + ResourceAmount max = interval.max[resource_type]; + if (max == Quota::UNLIMITED) + continue; + if (used > max) + { + bool used_counters_reset = false; + std::chrono::system_clock::time_point end_of_interval = getEndOfInterval(interval, current_time, &used_counters_reset); + if (!used_counters_reset) + throwQuotaExceed(user_name, intervals.quota_name, resource_type, used, max, interval.duration, end_of_interval); + } + } + } + + static void checkExceeded( + const String & user_name, + const Intervals & intervals, + std::chrono::system_clock::time_point current_time) + { + for (auto resource_type : ext::range_with_static_cast(Quota::MAX_RESOURCE_TYPE)) + checkExceeded(user_name, intervals, resource_type, current_time); + } +}; + + +QuotaContext::Interval & QuotaContext::Interval::operator =(const Interval & src) +{ + randomize_interval = src.randomize_interval; + duration = src.duration; + end_of_interval.store(src.end_of_interval.load()); + for (auto resource_type : ext::range(MAX_RESOURCE_TYPE)) + { + max[resource_type] = src.max[resource_type]; + used[resource_type].store(src.used[resource_type].load()); + } + return *this; +} + + +QuotaUsageInfo QuotaContext::Intervals::getUsageInfo(std::chrono::system_clock::time_point current_time) const +{ + QuotaUsageInfo info; + info.quota_id = quota_id; + info.quota_name = quota_name; + info.quota_key = quota_key; + info.intervals.reserve(intervals.size()); + for (const auto & in : intervals) + { + info.intervals.push_back({}); + auto & out = info.intervals.back(); + out.duration = in.duration; + out.randomize_interval = in.randomize_interval; + out.end_of_interval = Impl::getEndOfInterval(in, current_time); + for (auto resource_type : ext::range(MAX_RESOURCE_TYPE)) + { + out.max[resource_type] = in.max[resource_type]; + out.used[resource_type] = in.used[resource_type]; + } + } + return info; +} + + +QuotaContext::QuotaContext() + : atomic_intervals(std::make_shared()) /// Unlimited quota. +{ +} + + +QuotaContext::QuotaContext( + const String & user_name_, + const Poco::Net::IPAddress & address_, + const String & client_key_) + : user_name(user_name_), address(address_), client_key(client_key_) +{ +} + + +QuotaContext::~QuotaContext() = default; + + +void QuotaContext::used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded) +{ + used({resource_type, amount}, check_exceeded); +} + + +void QuotaContext::used(const std::pair & resource, bool check_exceeded) +{ + auto intervals_ptr = std::atomic_load(&atomic_intervals); + auto current_time = std::chrono::system_clock::now(); + Impl::used(user_name, *intervals_ptr, resource.first, resource.second, current_time, check_exceeded); +} + + +void QuotaContext::used(const std::pair & resource1, const std::pair & resource2, bool check_exceeded) +{ + auto intervals_ptr = std::atomic_load(&atomic_intervals); + auto current_time = std::chrono::system_clock::now(); + Impl::used(user_name, *intervals_ptr, resource1.first, resource1.second, current_time, check_exceeded); + Impl::used(user_name, *intervals_ptr, resource2.first, resource2.second, current_time, check_exceeded); +} + + +void QuotaContext::used(const std::pair & resource1, const std::pair & resource2, const std::pair & resource3, bool check_exceeded) +{ + auto intervals_ptr = std::atomic_load(&atomic_intervals); + auto current_time = std::chrono::system_clock::now(); + Impl::used(user_name, *intervals_ptr, resource1.first, resource1.second, current_time, check_exceeded); + Impl::used(user_name, *intervals_ptr, resource2.first, resource2.second, current_time, check_exceeded); + Impl::used(user_name, *intervals_ptr, resource3.first, resource3.second, current_time, check_exceeded); +} + + +void QuotaContext::used(const std::vector> & resources, bool check_exceeded) +{ + auto intervals_ptr = std::atomic_load(&atomic_intervals); + auto current_time = std::chrono::system_clock::now(); + for (const auto & resource : resources) + Impl::used(user_name, *intervals_ptr, resource.first, resource.second, current_time, check_exceeded); +} + + +void QuotaContext::checkExceeded() +{ + auto intervals_ptr = std::atomic_load(&atomic_intervals); + Impl::checkExceeded(user_name, *intervals_ptr, std::chrono::system_clock::now()); +} + + +void QuotaContext::checkExceeded(ResourceType resource_type) +{ + auto intervals_ptr = std::atomic_load(&atomic_intervals); + Impl::checkExceeded(user_name, *intervals_ptr, resource_type, std::chrono::system_clock::now()); +} + + +QuotaUsageInfo QuotaContext::getUsageInfo() const +{ + auto intervals_ptr = std::atomic_load(&atomic_intervals); + return intervals_ptr->getUsageInfo(std::chrono::system_clock::now()); +} + + +QuotaUsageInfo::QuotaUsageInfo() : quota_id(UUID(UInt128(0))) +{ +} + + +QuotaUsageInfo::Interval::Interval() +{ + boost::range::fill(used, 0); + boost::range::fill(max, 0); +} +} diff --git a/dbms/src/Access/QuotaContext.h b/dbms/src/Access/QuotaContext.h new file mode 100644 index 00000000000..122d0df6ee7 --- /dev/null +++ b/dbms/src/Access/QuotaContext.h @@ -0,0 +1,110 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +struct QuotaUsageInfo; + + +/// Instances of `QuotaContext` are used to track resource consumption. +class QuotaContext : public boost::noncopyable +{ +public: + using ResourceType = Quota::ResourceType; + using ResourceAmount = Quota::ResourceAmount; + + /// Default constructors makes an unlimited quota. + QuotaContext(); + + ~QuotaContext(); + + /// Tracks resource consumption. If the quota exceeded and `check_exceeded == true`, throws an exception. + void used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded = true); + void used(const std::pair & resource, bool check_exceeded = true); + void used(const std::pair & resource1, const std::pair & resource2, bool check_exceeded = true); + void used(const std::pair & resource1, const std::pair & resource2, const std::pair & resource3, bool check_exceeded = true); + void used(const std::vector> & resources, bool check_exceeded = true); + + /// Checks if the quota exceeded. If so, throws an exception. + void checkExceeded(); + void checkExceeded(ResourceType resource_type); + + /// Returns the information about this quota context. + QuotaUsageInfo getUsageInfo() const; + +private: + friend class QuotaContextFactory; + friend struct ext::shared_ptr_helper; + + /// Instances of this class are created by QuotaContextFactory. + QuotaContext(const String & user_name_, const Poco::Net::IPAddress & address_, const String & client_key_); + + static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; + + struct Interval + { + mutable std::atomic used[MAX_RESOURCE_TYPE]; + ResourceAmount max[MAX_RESOURCE_TYPE]; + std::chrono::seconds duration; + bool randomize_interval; + mutable std::atomic end_of_interval; + + Interval() {} + Interval(const Interval & src) { *this = src; } + Interval & operator =(const Interval & src); + }; + + struct Intervals + { + std::vector intervals; + UUID quota_id; + String quota_name; + String quota_key; + + QuotaUsageInfo getUsageInfo(std::chrono::system_clock::time_point current_time) const; + }; + + struct Impl; + + const String user_name; + const Poco::Net::IPAddress address; + const String client_key; + std::shared_ptr atomic_intervals; /// atomically changed by QuotaUsageManager +}; + +using QuotaContextPtr = std::shared_ptr; + + +/// The information about a quota context. +struct QuotaUsageInfo +{ + using ResourceType = Quota::ResourceType; + using ResourceAmount = Quota::ResourceAmount; + static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; + + struct Interval + { + ResourceAmount used[MAX_RESOURCE_TYPE]; + ResourceAmount max[MAX_RESOURCE_TYPE]; + std::chrono::seconds duration = std::chrono::seconds::zero(); + bool randomize_interval = false; + std::chrono::system_clock::time_point end_of_interval; + Interval(); + }; + + std::vector intervals; + UUID quota_id; + String quota_name; + String quota_key; + QuotaUsageInfo(); +}; +} diff --git a/dbms/src/Access/QuotaContextFactory.cpp b/dbms/src/Access/QuotaContextFactory.cpp new file mode 100644 index 00000000000..c6ecb947102 --- /dev/null +++ b/dbms/src/Access/QuotaContextFactory.cpp @@ -0,0 +1,299 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int QUOTA_REQUIRES_CLIENT_KEY; +} + + +namespace +{ + std::chrono::system_clock::duration randomDuration(std::chrono::seconds max) + { + auto count = std::chrono::duration_cast(max).count(); + std::uniform_int_distribution distribution{0, count - 1}; + return std::chrono::system_clock::duration(distribution(thread_local_rng)); + } +} + + +void QuotaContextFactory::QuotaInfo::setQuota(const QuotaPtr & quota_, const UUID & quota_id_) +{ + quota = quota_; + quota_id = quota_id_; + + boost::range::copy(quota->roles, std::inserter(roles, roles.end())); + all_roles = quota->all_roles; + boost::range::copy(quota->except_roles, std::inserter(except_roles, except_roles.end())); + + rebuildAllIntervals(); +} + + +bool QuotaContextFactory::QuotaInfo::canUseWithContext(const QuotaContext & context) const +{ + if (roles.count(context.user_name)) + return true; + + if (all_roles && !except_roles.count(context.user_name)) + return true; + + return false; +} + + +String QuotaContextFactory::QuotaInfo::calculateKey(const QuotaContext & context) const +{ + using KeyType = Quota::KeyType; + switch (quota->key_type) + { + case KeyType::NONE: + return ""; + case KeyType::USER_NAME: + return context.user_name; + case KeyType::IP_ADDRESS: + return context.address.toString(); + case KeyType::CLIENT_KEY: + { + if (!context.client_key.empty()) + return context.client_key; + throw Exception( + "Quota " + quota->getName() + " (for user " + context.user_name + ") requires a client supplied key.", + ErrorCodes::QUOTA_REQUIRES_CLIENT_KEY); + } + case KeyType::CLIENT_KEY_OR_USER_NAME: + { + if (!context.client_key.empty()) + return context.client_key; + return context.user_name; + } + case KeyType::CLIENT_KEY_OR_IP_ADDRESS: + { + if (!context.client_key.empty()) + return context.client_key; + return context.address.toString(); + } + } + __builtin_unreachable(); +} + + +std::shared_ptr QuotaContextFactory::QuotaInfo::getOrBuildIntervals(const String & key) +{ + auto it = key_to_intervals.find(key); + if (it != key_to_intervals.end()) + return it->second; + return rebuildIntervals(key); +} + + +void QuotaContextFactory::QuotaInfo::rebuildAllIntervals() +{ + for (const String & key : key_to_intervals | boost::adaptors::map_keys) + rebuildIntervals(key); +} + + +std::shared_ptr QuotaContextFactory::QuotaInfo::rebuildIntervals(const String & key) +{ + auto new_intervals = std::make_shared(); + new_intervals->quota_name = quota->getName(); + new_intervals->quota_id = quota_id; + new_intervals->quota_key = key; + auto & intervals = new_intervals->intervals; + intervals.reserve(quota->all_limits.size()); + constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; + for (const auto & limits : quota->all_limits) + { + intervals.emplace_back(); + auto & interval = intervals.back(); + interval.duration = limits.duration; + std::chrono::system_clock::time_point end_of_interval{}; + interval.randomize_interval = limits.randomize_interval; + if (limits.randomize_interval) + end_of_interval += randomDuration(limits.duration); + interval.end_of_interval = end_of_interval.time_since_epoch(); + for (auto resource_type : ext::range(MAX_RESOURCE_TYPE)) + { + interval.max[resource_type] = limits.max[resource_type]; + interval.used[resource_type] = 0; + } + } + + /// Order intervals by durations from largest to smallest. + /// To report first about largest interval on what quota was exceeded. + struct GreaterByDuration + { + bool operator()(const Interval & lhs, const Interval & rhs) const { return lhs.duration > rhs.duration; } + }; + boost::range::stable_sort(intervals, GreaterByDuration{}); + + auto it = key_to_intervals.find(key); + if (it == key_to_intervals.end()) + { + /// Just put new intervals into the map. + key_to_intervals.try_emplace(key, new_intervals); + } + else + { + /// We need to keep usage information from the old intervals. + const auto & old_intervals = it->second->intervals; + for (auto & new_interval : new_intervals->intervals) + { + /// Check if an interval with the same duration is already in use. + auto lower_bound = boost::range::lower_bound(old_intervals, new_interval, GreaterByDuration{}); + if ((lower_bound == old_intervals.end()) || (lower_bound->duration != new_interval.duration)) + continue; + + /// Found an interval with the same duration, we need to copy its usage information to `result`. + auto & current_interval = *lower_bound; + for (auto resource_type : ext::range(MAX_RESOURCE_TYPE)) + { + new_interval.used[resource_type].store(current_interval.used[resource_type].load()); + new_interval.end_of_interval.store(current_interval.end_of_interval.load()); + } + } + it->second = new_intervals; + } + + return new_intervals; +} + + +QuotaContextFactory::QuotaContextFactory(const AccessControlManager & access_control_manager_) + : access_control_manager(access_control_manager_) +{ +} + + +QuotaContextFactory::~QuotaContextFactory() +{ +} + + +std::shared_ptr QuotaContextFactory::createContext(const String & user_name, const Poco::Net::IPAddress & address, const String & client_key) +{ + std::lock_guard lock{mutex}; + ensureAllQuotasRead(); + auto context = ext::shared_ptr_helper::create(user_name, address, client_key); + contexts.push_back(context); + chooseQuotaForContext(context); + return context; +} + + +void QuotaContextFactory::ensureAllQuotasRead() +{ + /// `mutex` is already locked. + if (all_quotas_read) + return; + all_quotas_read = true; + + subscription = access_control_manager.subscribeForChanges( + [&](const UUID & id, const AccessEntityPtr & entity) + { + if (entity) + quotaAddedOrChanged(id, typeid_cast(entity)); + else + quotaRemoved(id); + }); + + for (const UUID & quota_id : access_control_manager.findAll()) + { + auto quota = access_control_manager.tryRead(quota_id); + if (quota) + all_quotas.emplace(quota_id, QuotaInfo(quota, quota_id)); + } +} + + +void QuotaContextFactory::quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr & new_quota) +{ + std::lock_guard lock{mutex}; + auto it = all_quotas.find(quota_id); + if (it == all_quotas.end()) + { + it = all_quotas.emplace(quota_id, QuotaInfo(new_quota, quota_id)).first; + } + else + { + if (it->second.quota == new_quota) + return; + } + + auto & info = it->second; + info.setQuota(new_quota, quota_id); + chooseQuotaForAllContexts(); +} + + +void QuotaContextFactory::quotaRemoved(const UUID & quota_id) +{ + std::lock_guard lock{mutex}; + all_quotas.erase(quota_id); + chooseQuotaForAllContexts(); +} + + +void QuotaContextFactory::chooseQuotaForAllContexts() +{ + /// `mutex` is already locked. + boost::range::remove_erase_if( + contexts, + [&](const std::weak_ptr & weak) + { + auto context = weak.lock(); + if (!context) + return true; // remove from the `contexts` list. + chooseQuotaForContext(context); + return false; // keep in the `contexts` list. + }); +} + +void QuotaContextFactory::chooseQuotaForContext(const std::shared_ptr & context) +{ + /// `mutex` is already locked. + std::shared_ptr intervals; + for (auto & info : all_quotas | boost::adaptors::map_values) + { + if (info.canUseWithContext(*context)) + { + String key = info.calculateKey(*context); + intervals = info.getOrBuildIntervals(key); + break; + } + } + + if (!intervals) + intervals = std::make_shared(); /// No quota == no limits. + + std::atomic_store(&context->atomic_intervals, intervals); +} + + +std::vector QuotaContextFactory::getUsageInfo() const +{ + std::lock_guard lock{mutex}; + std::vector all_infos; + auto current_time = std::chrono::system_clock::now(); + for (const auto & info : all_quotas | boost::adaptors::map_values) + { + for (const auto & intervals : info.key_to_intervals | boost::adaptors::map_values) + all_infos.push_back(intervals->getUsageInfo(current_time)); + } + return all_infos; +} +} diff --git a/dbms/src/Access/QuotaContextFactory.h b/dbms/src/Access/QuotaContextFactory.h new file mode 100644 index 00000000000..159ffe1fa09 --- /dev/null +++ b/dbms/src/Access/QuotaContextFactory.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +class AccessControlManager; + + +/// Stores information how much amount of resources have been consumed and how much are left. +class QuotaContextFactory +{ +public: + QuotaContextFactory(const AccessControlManager & access_control_manager_); + ~QuotaContextFactory(); + + QuotaContextPtr createContext(const String & user_name, const Poco::Net::IPAddress & address, const String & client_key); + std::vector getUsageInfo() const; + +private: + using Interval = QuotaContext::Interval; + using Intervals = QuotaContext::Intervals; + + struct QuotaInfo + { + QuotaInfo(const QuotaPtr & quota_, const UUID & quota_id_) { setQuota(quota_, quota_id_); } + void setQuota(const QuotaPtr & quota_, const UUID & quota_id_); + + bool canUseWithContext(const QuotaContext & context) const; + String calculateKey(const QuotaContext & context) const; + std::shared_ptr getOrBuildIntervals(const String & key); + std::shared_ptr rebuildIntervals(const String & key); + void rebuildAllIntervals(); + + QuotaPtr quota; + UUID quota_id; + std::unordered_set roles; + bool all_roles = false; + std::unordered_set except_roles; + std::unordered_map> key_to_intervals; + }; + + void ensureAllQuotasRead(); + void quotaAddedOrChanged(const UUID & quota_id, const std::shared_ptr & new_quota); + void quotaRemoved(const UUID & quota_id); + void chooseQuotaForAllContexts(); + void chooseQuotaForContext(const std::shared_ptr & context); + + const AccessControlManager & access_control_manager; + mutable std::mutex mutex; + std::unordered_map all_quotas; + bool all_quotas_read = false; + IAccessStorage::SubscriptionPtr subscription; + std::vector> contexts; +}; +} diff --git a/dbms/src/Access/UsersConfigAccessStorage.cpp b/dbms/src/Access/UsersConfigAccessStorage.cpp index caabb9ff34e..d417968bb64 100644 --- a/dbms/src/Access/UsersConfigAccessStorage.cpp +++ b/dbms/src/Access/UsersConfigAccessStorage.cpp @@ -1,5 +1,7 @@ #include -#include +#include +#include +#include #include #include #include @@ -9,10 +11,10 @@ namespace DB { namespace { -#if 0 char getTypeChar(std::type_index type) { - UNUSED(type); /// TODO + if (type == typeid(Quota)) + return 'Q'; return 0; } @@ -31,7 +33,85 @@ namespace UUID generateID(const IAccessEntity & entity) { return generateID(entity.getType(), entity.getFullName()); } -#endif + + QuotaPtr parseQuota(const Poco::Util::AbstractConfiguration & config, const String & quota_name, const Strings & user_names) + { + auto quota = std::make_shared(); + quota->setName(quota_name); + + using KeyType = Quota::KeyType; + String quota_config = "quotas." + quota_name; + if (config.has(quota_config + ".keyed_by_ip")) + quota->key_type = KeyType::IP_ADDRESS; + else if (config.has(quota_config + ".keyed")) + quota->key_type = KeyType::CLIENT_KEY_OR_USER_NAME; + else + quota->key_type = KeyType::USER_NAME; + + Poco::Util::AbstractConfiguration::Keys interval_keys; + config.keys(quota_config, interval_keys); + + for (const String & interval_key : interval_keys) + { + if (!startsWith(interval_key, "interval")) + continue; + + String interval_config = quota_config + "." + interval_key; + std::chrono::seconds duration{config.getInt(interval_config + ".duration", 0)}; + if (duration.count() <= 0) /// Skip quotas with non-positive duration. + continue; + + quota->all_limits.emplace_back(); + auto & limits = quota->all_limits.back(); + limits.duration = duration; + limits.randomize_interval = config.getBool(interval_config + ".randomize", false); + + using ResourceType = Quota::ResourceType; + limits.max[ResourceType::QUERIES] = config.getUInt64(interval_config + ".queries", Quota::UNLIMITED); + limits.max[ResourceType::ERRORS] = config.getUInt64(interval_config + ".errors", Quota::UNLIMITED); + limits.max[ResourceType::RESULT_ROWS] = config.getUInt64(interval_config + ".result_rows", Quota::UNLIMITED); + limits.max[ResourceType::RESULT_BYTES] = config.getUInt64(interval_config + ".result_bytes", Quota::UNLIMITED); + limits.max[ResourceType::READ_ROWS] = config.getUInt64(interval_config + ".read_rows", Quota::UNLIMITED); + limits.max[ResourceType::READ_BYTES] = config.getUInt64(interval_config + ".read_bytes", Quota::UNLIMITED); + limits.max[ResourceType::EXECUTION_TIME] = Quota::secondsToExecutionTime(config.getUInt64(interval_config + ".execution_time", Quota::UNLIMITED)); + } + + quota->roles = user_names; + + return quota; + } + + + std::vector parseQuotas(const Poco::Util::AbstractConfiguration & config, Poco::Logger * log) + { + Poco::Util::AbstractConfiguration::Keys user_names; + config.keys("users", user_names); + std::unordered_map quota_to_user_names; + for (const auto & user_name : user_names) + { + if (config.has("users." + user_name + ".quota")) + quota_to_user_names[config.getString("users." + user_name + ".quota")].push_back(user_name); + } + + Poco::Util::AbstractConfiguration::Keys quota_names; + config.keys("quotas", quota_names); + std::vector quotas; + quotas.reserve(quota_names.size()); + for (const auto & quota_name : quota_names) + { + try + { + auto it = quota_to_user_names.find(quota_name); + const Strings quota_users = (it != quota_to_user_names.end()) ? std::move(it->second) : Strings{}; + quotas.push_back(parseQuota(config, quota_name, quota_users)); + } + catch (...) + { + tryLogCurrentException(log, "Could not parse quota " + backQuote(quota_name)); + } + } + return quotas; + } } @@ -46,7 +126,8 @@ UsersConfigAccessStorage::~UsersConfigAccessStorage() {} void UsersConfigAccessStorage::loadFromConfig(const Poco::Util::AbstractConfiguration & config) { std::vector> all_entities; - UNUSED(config); /// TODO + for (const auto & entity : parseQuotas(config, getLogger())) + all_entities.emplace_back(generateID(*entity), entity); memory_storage.setAll(all_entities); } diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 0258bcdb6b0..5fba98b7a69 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -470,6 +470,7 @@ namespace ErrorCodes extern const int ACCESS_ENTITY_ALREADY_EXISTS = 493; extern const int ACCESS_ENTITY_FOUND_DUPLICATES = 494; extern const int ACCESS_ENTITY_STORAGE_READONLY = 495; + extern const int QUOTA_REQUIRES_CLIENT_KEY = 496; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/DataStreams/IBlockInputStream.cpp b/dbms/src/DataStreams/IBlockInputStream.cpp index 2e30749e89f..df81f26f665 100644 --- a/dbms/src/DataStreams/IBlockInputStream.cpp +++ b/dbms/src/DataStreams/IBlockInputStream.cpp @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -70,7 +70,7 @@ Block IBlockInputStream::read() if (limits.mode == LIMITS_CURRENT && !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) limit_exceeded_need_break = true; - if (quota != nullptr) + if (quota) checkQuota(res); } else @@ -240,12 +240,8 @@ void IBlockInputStream::checkQuota(Block & block) case LIMITS_CURRENT: { - time_t current_time = time(nullptr); - double total_elapsed = info.total_stopwatch.elapsedSeconds(); - - quota->checkAndAddResultRowsBytes(current_time, block.rows(), block.bytes()); - quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0)); - + UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds(); + quota->used({Quota::RESULT_ROWS, block.rows()}, {Quota::RESULT_BYTES, block.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); prev_elapsed = total_elapsed; break; } @@ -291,10 +287,8 @@ void IBlockInputStream::progressImpl(const Progress & value) limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); - if (quota != nullptr && limits.mode == LIMITS_TOTAL) - { - quota->checkAndAddReadRowsBytes(time(nullptr), value.read_rows, value.read_bytes); - } + if (quota && limits.mode == LIMITS_TOTAL) + quota->used({Quota::READ_ROWS, value.read_rows}, {Quota::READ_BYTES, value.read_bytes}); } } diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index dfa9194a6f9..69aadf44c09 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -23,7 +23,7 @@ namespace ErrorCodes } class ProcessListElement; -class QuotaForIntervals; +class QuotaContext; class QueryStatus; struct SortColumnDescription; using SortDescription = std::vector; @@ -220,9 +220,9 @@ public: /** Set the quota. If you set a quota on the amount of raw data, * then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. */ - virtual void setQuota(QuotaForIntervals & quota_) + virtual void setQuota(const std::shared_ptr & quota_) { - quota = "a_; + quota = quota_; } /// Enable calculation of minimums and maximums by the result columns. @@ -273,8 +273,8 @@ private: LocalLimits limits; - QuotaForIntervals * quota = nullptr; /// If nullptr - the quota is not used. - double prev_elapsed = 0; + std::shared_ptr quota; /// If nullptr - the quota is not used. + UInt64 prev_elapsed = 0; /// The approximate total number of rows to read. For progress bar. size_t total_rows_approx = 0; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index c1a41a72507..1929ba29cce 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -26,9 +27,9 @@ #include #include #include +#include #include #include -#include #include #include #include @@ -133,7 +134,6 @@ struct ContextShared String system_profile_name; /// Profile used by system processes AccessControlManager access_control_manager; std::unique_ptr users_manager; /// Known users. - Quotas quotas; /// Known quotas for resource use. mutable UncompressedCachePtr uncompressed_cache; /// The cache of decompressed blocks. mutable MarkCachePtr mark_cache; /// Cache of marks in compressed files. ProcessList process_list; /// Executing queries at the moment. @@ -328,7 +328,7 @@ Context & Context::operator=(const Context &) = default; Context Context::createGlobal() { Context res; - res.quota = std::make_shared(); + res.quota = std::make_shared(); res.shared = std::make_shared(); return res; } @@ -605,7 +605,6 @@ void Context::setUsersConfig(const ConfigurationPtr & config) shared->users_config = config; shared->access_control_manager.loadFromConfig(*shared->users_config); shared->users_manager->loadFromConfig(*shared->users_config); - shared->quotas.loadFromConfig(*shared->users_config); } ConfigurationPtr Context::getUsersConfig() @@ -646,7 +645,8 @@ void Context::calculateUserSettings() { auto lock = getLock(); - String profile = shared->users_manager->getUser(client_info.current_user)->profile; + auto user = getUser(client_info.current_user); + String profile = user->profile; /// 1) Set default settings (hardcoded values) /// NOTE: we ignore global_context settings (from which it is usually copied) @@ -661,6 +661,9 @@ void Context::calculateUserSettings() /// 3) Apply settings from current user setProfile(profile); + + quota = getAccessControlManager().createQuotaContext( + client_info.current_user, client_info.current_address.host(), client_info.quota_key); } @@ -693,24 +696,9 @@ void Context::setUser(const String & name, const String & password, const Poco:: client_info.quota_key = quota_key; calculateUserSettings(); - - setQuota(user_props->quota, quota_key, name, address.host()); } -void Context::setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address) -{ - auto lock = getLock(); - quota = shared->quotas.get(name, quota_key, user_name, address); -} - - -QuotaForIntervals & Context::getQuota() -{ - auto lock = getLock(); - return *quota; -} - void Context::checkDatabaseAccessRights(const std::string & database_name) const { auto lock = getLock(); diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index eb561fd92a8..a204aab5bc5 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -44,7 +44,7 @@ namespace DB struct ContextShared; class Context; -class QuotaForIntervals; +class QuotaContext; class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalModelsLoader; @@ -138,7 +138,7 @@ private: InputInitializer input_initializer_callback; InputBlocksReader input_blocks_reader; - std::shared_ptr quota; /// Current quota. By default - empty quota, that have no limits. + std::shared_ptr quota; /// Current quota. By default - empty quota, that have no limits. String current_database; Settings settings; /// Setting for query execution. std::shared_ptr settings_constraints; @@ -204,6 +204,7 @@ public: AccessControlManager & getAccessControlManager(); const AccessControlManager & getAccessControlManager() const; + std::shared_ptr getQuota() const { return quota; } /** Take the list of users, quotas and configuration profiles from this config. * The list of users is completely replaced. @@ -244,9 +245,6 @@ public: ClientInfo & getClientInfo() { return client_info; } const ClientInfo & getClientInfo() const { return client_info; } - void setQuota(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & address); - QuotaForIntervals & getQuota(); - void addDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where); void removeDependency(const DatabaseAndTableName & from, const DatabaseAndTableName & where); Dependencies getDependencies(const String & database_name, const String & table_name) const; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 0f3d5d82f96..7134b52d7cd 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1776,7 +1776,7 @@ void InterpreterSelectQuery::executeFetchColumns( limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; } - QuotaForIntervals & quota = context->getQuota(); + auto quota = context->getQuota(); for (auto & stream : streams) { diff --git a/dbms/src/Interpreters/Quota.cpp b/dbms/src/Interpreters/Quota.cpp deleted file mode 100644 index 5123f4fd3e8..00000000000 --- a/dbms/src/Interpreters/Quota.cpp +++ /dev/null @@ -1,345 +0,0 @@ -#include - -#include - -#include -#include -#include -#include - -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int QUOTA_EXPIRED; - extern const int QUOTA_DOESNT_ALLOW_KEYS; - extern const int UNKNOWN_QUOTA; -} - - -template -void QuotaValues::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config) -{ - queries = config.getUInt64(config_elem + ".queries", 0); - errors = config.getUInt64(config_elem + ".errors", 0); - result_rows = config.getUInt64(config_elem + ".result_rows", 0); - result_bytes = config.getUInt64(config_elem + ".result_bytes", 0); - read_rows = config.getUInt64(config_elem + ".read_rows", 0); - read_bytes = config.getUInt64(config_elem + ".read_bytes", 0); - execution_time_usec = config.getUInt64(config_elem + ".execution_time", 0) * 1000000ULL; -} - -template void QuotaValues::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); -template void QuotaValues>::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); - - -void QuotaForInterval::initFromConfig( - const String & config_elem, time_t duration_, bool randomize_, time_t offset_, const Poco::Util::AbstractConfiguration & config) -{ - rounded_time.store(0, std::memory_order_relaxed); - duration = duration_; - randomize = randomize_; - offset = offset_; - max.initFromConfig(config_elem, config); -} - -void QuotaForInterval::checkExceeded(time_t current_time, const String & quota_name, const String & user_name) -{ - updateTime(current_time); - check(max.queries, used.queries, quota_name, user_name, "Queries"); - check(max.errors, used.errors, quota_name, user_name, "Errors"); - check(max.result_rows, used.result_rows, quota_name, user_name, "Total result rows"); - check(max.result_bytes, used.result_bytes, quota_name, user_name, "Total result bytes"); - check(max.read_rows, used.read_rows, quota_name, user_name, "Total rows read"); - check(max.read_bytes, used.read_bytes, quota_name, user_name, "Total bytes read"); - check(max.execution_time_usec / 1000000, used.execution_time_usec / 1000000, quota_name, user_name, "Total execution time"); -} - -String QuotaForInterval::toString() const -{ - std::stringstream res; - - auto loaded_rounded_time = rounded_time.load(std::memory_order_relaxed); - - res << std::fixed << std::setprecision(3) - << "Interval: " << LocalDateTime(loaded_rounded_time) << " - " << LocalDateTime(loaded_rounded_time + duration) << ".\n" - << "Queries: " << used.queries << ".\n" - << "Errors: " << used.errors << ".\n" - << "Result rows: " << used.result_rows << ".\n" - << "Result bytes: " << used.result_bytes << ".\n" - << "Read rows: " << used.read_rows << ".\n" - << "Read bytes: " << used.read_bytes << ".\n" - << "Execution time: " << used.execution_time_usec / 1000000.0 << " sec.\n"; - - return res.str(); -} - -void QuotaForInterval::addQuery() noexcept -{ - ++used.queries; -} - -void QuotaForInterval::addError() noexcept -{ - ++used.errors; -} - -void QuotaForInterval::checkAndAddResultRowsBytes(time_t current_time, const String & quota_name, const String & user_name, size_t rows, size_t bytes) -{ - used.result_rows += rows; - used.result_bytes += bytes; - checkExceeded(current_time, quota_name, user_name); -} - -void QuotaForInterval::checkAndAddReadRowsBytes(time_t current_time, const String & quota_name, const String & user_name, size_t rows, size_t bytes) -{ - used.read_rows += rows; - used.read_bytes += bytes; - checkExceeded(current_time, quota_name, user_name); -} - -void QuotaForInterval::checkAndAddExecutionTime(time_t current_time, const String & quota_name, const String & user_name, Poco::Timespan amount) -{ - /// Information about internals of Poco::Timespan used. - used.execution_time_usec += amount.totalMicroseconds(); - checkExceeded(current_time, quota_name, user_name); -} - -void QuotaForInterval::updateTime(time_t current_time) -{ - /** If current time is greater than end of interval, - * then clear accumulated quota values and switch to next interval [rounded_time, rounded_time + duration). - */ - - auto loaded_rounded_time = rounded_time.load(std::memory_order_acquire); - while (true) - { - if (current_time < loaded_rounded_time + static_cast(duration)) - break; - - time_t new_rounded_time = (current_time - offset) / duration * duration + offset; - if (rounded_time.compare_exchange_strong(loaded_rounded_time, new_rounded_time)) - { - used.clear(); - break; - } - } -} - -void QuotaForInterval::check( - size_t max_amount, size_t used_amount, - const String & quota_name, const String & user_name, const char * resource_name) -{ - if (max_amount && used_amount > max_amount) - { - std::stringstream message; - message << "Quota for user '" << user_name << "' for "; - - if (duration == 3600) - message << "1 hour"; - else if (duration == 60) - message << "1 minute"; - else if (duration % 3600 == 0) - message << (duration / 3600) << " hours"; - else if (duration % 60 == 0) - message << (duration / 60) << " minutes"; - else - message << duration << " seconds"; - - message << " has been exceeded. " - << resource_name << ": " << used_amount << ", max: " << max_amount << ". " - << "Interval will end at " << LocalDateTime(rounded_time.load(std::memory_order_relaxed) + duration) << ". " - << "Name of quota template: '" << quota_name << "'."; - - throw Exception(message.str(), ErrorCodes::QUOTA_EXPIRED); - } -} - - -void QuotaForIntervals::initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, pcg64 & rng) -{ - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys(config_elem, config_keys); - - for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it) - { - if (!startsWith(*it, "interval")) - continue; - - String interval_config_elem = config_elem + "." + *it; - time_t duration = config.getInt(interval_config_elem + ".duration", 0); - time_t offset = 0; - - if (!duration) /// Skip quotas with zero duration - continue; - - bool randomize = config.getBool(interval_config_elem + ".randomize", false); - if (randomize) - offset = std::uniform_int_distribution(0, duration - 1)(rng); - - cont[duration].initFromConfig(interval_config_elem, duration, randomize, offset, config); - } -} - -void QuotaForIntervals::setMax(const QuotaForIntervals & quota) -{ - for (Container::iterator it = cont.begin(); it != cont.end();) - { - if (quota.cont.count(it->first)) - ++it; - else - cont.erase(it++); - } - - for (auto & x : quota.cont) - { - if (!cont.count(x.first)) - cont.emplace(x.first, x.second); - else - cont[x.first].max = x.second.max; - } -} - -void QuotaForIntervals::checkExceeded(time_t current_time) -{ - for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) - it->second.checkExceeded(current_time, quota_name, user_name); -} - -void QuotaForIntervals::addQuery() noexcept -{ - for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) - it->second.addQuery(); -} - -void QuotaForIntervals::addError() noexcept -{ - for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) - it->second.addError(); -} - -void QuotaForIntervals::checkAndAddResultRowsBytes(time_t current_time, size_t rows, size_t bytes) -{ - for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) - it->second.checkAndAddResultRowsBytes(current_time, quota_name, user_name, rows, bytes); -} - -void QuotaForIntervals::checkAndAddReadRowsBytes(time_t current_time, size_t rows, size_t bytes) -{ - for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) - it->second.checkAndAddReadRowsBytes(current_time, quota_name, user_name, rows, bytes); -} - -void QuotaForIntervals::checkAndAddExecutionTime(time_t current_time, Poco::Timespan amount) -{ - for (Container::reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) - it->second.checkAndAddExecutionTime(current_time, quota_name, user_name, amount); -} - -String QuotaForIntervals::toString() const -{ - std::stringstream res; - - for (Container::const_reverse_iterator it = cont.rbegin(); it != cont.rend(); ++it) - res << std::endl << it->second.toString(); - - return res.str(); -} - - -void Quota::loadFromConfig(const String & config_elem, const String & name_, const Poco::Util::AbstractConfiguration & config, pcg64 & rng) -{ - name = name_; - - bool new_keyed_by_ip = config.has(config_elem + ".keyed_by_ip"); - bool new_is_keyed = new_keyed_by_ip || config.has(config_elem + ".keyed"); - - if (new_is_keyed != is_keyed || new_keyed_by_ip != keyed_by_ip) - { - keyed_by_ip = new_keyed_by_ip; - is_keyed = new_is_keyed; - /// Meaning of keys has been changed. Throw away accumulated values. - quota_for_keys.clear(); - } - - ignore_key_if_not_keyed = config.has(config_elem + ".ignore_key_if_not_keyed"); - - QuotaForIntervals new_max(name, {}); - new_max.initFromConfig(config_elem, config, rng); - if (!new_max.hasEqualConfiguration(max)) - { - max = new_max; - for (auto & quota : quota_for_keys) - quota.second->setMax(max); - } -} - -QuotaForIntervalsPtr Quota::get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip) -{ - if (!quota_key.empty() && !ignore_key_if_not_keyed && (!is_keyed || keyed_by_ip)) - throw Exception("Quota " + name + " (for user " + user_name + ") doesn't allow client supplied keys.", - ErrorCodes::QUOTA_DOESNT_ALLOW_KEYS); - - /** Quota is calculated separately: - * - for each IP-address, if 'keyed_by_ip'; - * - otherwise for each 'quota_key', if present; - * - otherwise for each 'user_name'. - */ - - UInt64 quota_key_hashed = sipHash64( - keyed_by_ip - ? ip.toString() - : (!quota_key.empty() - ? quota_key - : user_name)); - - std::lock_guard lock(mutex); - - Container::iterator it = quota_for_keys.find(quota_key_hashed); - if (quota_for_keys.end() == it) - it = quota_for_keys.emplace(quota_key_hashed, std::make_shared(max, user_name)).first; - - return it->second; -} - - -void Quotas::loadFromConfig(const Poco::Util::AbstractConfiguration & config) -{ - pcg64 rng; - - Poco::Util::AbstractConfiguration::Keys config_keys; - config.keys("quotas", config_keys); - - /// Remove keys, that now absent in config. - std::set keys_set(config_keys.begin(), config_keys.end()); - for (Container::iterator it = cont.begin(); it != cont.end();) - { - if (keys_set.count(it->first)) - ++it; - else - cont.erase(it++); - } - - for (Poco::Util::AbstractConfiguration::Keys::const_iterator it = config_keys.begin(); it != config_keys.end(); ++it) - { - if (!cont.count(*it)) - cont.try_emplace(*it); - cont[*it].loadFromConfig("quotas." + *it, *it, config, rng); - } -} - -QuotaForIntervalsPtr Quotas::get(const String & name, const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip) -{ - Container::iterator it = cont.find(name); - if (cont.end() == it) - throw Exception("Unknown quota " + name, ErrorCodes::UNKNOWN_QUOTA); - - return it->second.get(quota_key, user_name, ip); -} - -} diff --git a/dbms/src/Interpreters/Quota.h b/dbms/src/Interpreters/Quota.h deleted file mode 100644 index c1fb3f143fb..00000000000 --- a/dbms/src/Interpreters/Quota.h +++ /dev/null @@ -1,263 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -#include - -#include -#include - -#include - -#include -#include -#include - - -namespace DB -{ - -/** Quota for resources consumption for specific interval. - * Used to limit resource usage by user. - * Quota is applied "softly" - could be slightly exceed, because it is checked usually only on each block of processed data. - * Accumulated values are not persisted and are lost on server restart. - * Quota is local to server, - * but for distributed queries, accumulated values for read rows and bytes - * are collected from all participating servers and accumulated locally. - */ - -/// Used both for maximum allowed values and for counters of current accumulated values. -template /// either size_t or std::atomic -struct QuotaValues -{ - /// Zero values (for maximums) means no limit. - Counter queries; /// Number of queries. - Counter errors; /// Number of queries with exceptions. - Counter result_rows; /// Number of rows returned as result. - Counter result_bytes; /// Number of bytes returned as result. - Counter read_rows; /// Number of rows read from tables. - Counter read_bytes; /// Number of bytes read from tables. - Counter execution_time_usec; /// Total amount of query execution time in microseconds. - - QuotaValues() - { - clear(); - } - - QuotaValues(const QuotaValues & rhs) - { - tuple() = rhs.tuple(); - } - - QuotaValues & operator=(const QuotaValues & rhs) - { - tuple() = rhs.tuple(); - return *this; - } - - void clear() - { - tuple() = std::make_tuple(0, 0, 0, 0, 0, 0, 0); - } - - void initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config); - - bool operator== (const QuotaValues & rhs) const - { - return tuple() == rhs.tuple(); - } - -private: - auto tuple() - { - return std::forward_as_tuple(queries, errors, result_rows, result_bytes, read_rows, read_bytes, execution_time_usec); - } - - auto tuple() const - { - return std::make_tuple(queries, errors, result_rows, result_bytes, read_rows, read_bytes, execution_time_usec); - } -}; - -template <> -inline auto QuotaValues>::tuple() const -{ - return std::make_tuple( - queries.load(std::memory_order_relaxed), - errors.load(std::memory_order_relaxed), - result_rows.load(std::memory_order_relaxed), - result_bytes.load(std::memory_order_relaxed), - read_rows.load(std::memory_order_relaxed), - read_bytes.load(std::memory_order_relaxed), - execution_time_usec.load(std::memory_order_relaxed)); -} - - -/// Time, rounded down to start of interval; limits for that interval and accumulated values. -struct QuotaForInterval -{ - std::atomic rounded_time {0}; - size_t duration = 0; - bool randomize = false; - time_t offset = 0; /// Offset of interval for randomization (to avoid DoS if intervals for many users end at one time). - QuotaValues max; - QuotaValues> used; - - QuotaForInterval() = default; - QuotaForInterval(time_t duration_) : duration(duration_) {} - - void initFromConfig(const String & config_elem, time_t duration_, bool randomize_, time_t offset_, const Poco::Util::AbstractConfiguration & config); - - /// Increase current value. - void addQuery() noexcept; - void addError() noexcept; - - /// Check if quota is already exceeded. If that, throw an exception. - void checkExceeded(time_t current_time, const String & quota_name, const String & user_name); - - /// Check corresponding value. If exceeded, throw an exception. Otherwise, increase that value. - void checkAndAddResultRowsBytes(time_t current_time, const String & quota_name, const String & user_name, size_t rows, size_t bytes); - void checkAndAddReadRowsBytes(time_t current_time, const String & quota_name, const String & user_name, size_t rows, size_t bytes); - void checkAndAddExecutionTime(time_t current_time, const String & quota_name, const String & user_name, Poco::Timespan amount); - - /// Get a text, describing what quota is exceeded. - String toString() const; - - /// Only compare configuration, not accumulated (used) values or random offsets. - bool operator== (const QuotaForInterval & rhs) const - { - return randomize == rhs.randomize - && duration == rhs.duration - && max == rhs.max; - } - - QuotaForInterval & operator= (const QuotaForInterval & rhs) - { - rounded_time.store(rhs.rounded_time.load(std::memory_order_relaxed)); - duration = rhs.duration; - randomize = rhs.randomize; - offset = rhs.offset; - max = rhs.max; - used = rhs.used; - return *this; - } - - QuotaForInterval(const QuotaForInterval & rhs) - { - *this = rhs; - } - -private: - /// Reset counters of used resources, if interval for quota is expired. - void updateTime(time_t current_time); - void check(size_t max_amount, size_t used_amount, - const String & quota_name, const String & user_name, const char * resource_name); -}; - - -struct Quota; - -/// Length of interval -> quota: maximum allowed and currently accumulated values for that interval (example: 3600 -> values for current hour). -class QuotaForIntervals -{ -private: - /// While checking, will walk through intervals in order of decreasing size - from largest to smallest. - /// To report first about largest interval on what quota was exceeded. - using Container = std::map; - Container cont; - - std::string quota_name; - std::string user_name; /// user name is set only for current counters for user, not for object that contain maximum values (limits). - -public: - QuotaForIntervals(const std::string & quota_name_, const std::string & user_name_) - : quota_name(quota_name_), user_name(user_name_) {} - - QuotaForIntervals(const QuotaForIntervals & other, const std::string & user_name_) - : QuotaForIntervals(other) - { - user_name = user_name_; - } - - QuotaForIntervals() = default; - QuotaForIntervals(const QuotaForIntervals &) = default; - QuotaForIntervals & operator=(const QuotaForIntervals &) = default; - - /// Is there at least one interval for counting quota? - bool empty() const - { - return cont.empty(); - } - - void initFromConfig(const String & config_elem, const Poco::Util::AbstractConfiguration & config, pcg64 & rng); - - /// Set maximum values (limits) from passed argument. - /// Remove intervals that does not exist in argument. Add intervals from argument, that we don't have. - void setMax(const QuotaForIntervals & quota); - - void addQuery() noexcept; - void addError() noexcept; - - void checkExceeded(time_t current_time); - - void checkAndAddResultRowsBytes(time_t current_time, size_t rows, size_t bytes); - void checkAndAddReadRowsBytes(time_t current_time, size_t rows, size_t bytes); - void checkAndAddExecutionTime(time_t current_time, Poco::Timespan amount); - - /// Get text, describing what part of quota has been exceeded. - String toString() const; - - bool hasEqualConfiguration(const QuotaForIntervals & rhs) const - { - return cont == rhs.cont && quota_name == rhs.quota_name; - } -}; - -using QuotaForIntervalsPtr = std::shared_ptr; - - -/// Quota key -> quotas (max and current values) for intervals. If quota doesn't have keys, then values stored at key 0. -struct Quota -{ - using Container = std::unordered_map; - - String name; - - /// Maximum values from config. - QuotaForIntervals max; - /// Maximum and accumulated values for different keys. - /// For all keys, maximum values are the same and taken from 'max'. - Container quota_for_keys; - std::mutex mutex; - - bool is_keyed = false; - - /// If the quota is not keyed, but the user passed some key, ignore it instead of throwing exception. - /// For transitional periods, when you want to enable quota keys - /// - first, enable passing keys from your application, then make quota keyed in ClickHouse users config. - bool ignore_key_if_not_keyed = false; - - bool keyed_by_ip = false; - - void loadFromConfig(const String & config_elem, const String & name_, const Poco::Util::AbstractConfiguration & config, pcg64 & rng); - QuotaForIntervalsPtr get(const String & quota_key, const String & user_name, const Poco::Net::IPAddress & ip); -}; - - -class Quotas -{ -private: - /// Name of quota -> quota. - using Container = std::unordered_map; - Container cont; - -public: - void loadFromConfig(const Poco::Util::AbstractConfiguration & config); - QuotaForIntervalsPtr get(const String & name, const String & quota_key, - const String & user_name, const Poco::Net::IPAddress & ip); -}; - -} diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index 2ca2873e95b..7ac738031c2 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -49,7 +49,6 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A } profile = config.getString(config_elem + ".profile"); - quota = config.getString(config_elem + ".quota"); /// Fill list of allowed hosts. const auto config_networks = config_elem + ".networks"; diff --git a/dbms/src/Interpreters/Users.h b/dbms/src/Interpreters/Users.h index e116772855a..b7340fadcfa 100644 --- a/dbms/src/Interpreters/Users.h +++ b/dbms/src/Interpreters/Users.h @@ -30,7 +30,6 @@ struct User Authentication authentication; String profile; - String quota; AllowedClientHosts allowed_client_hosts; diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 41c8e288ffe..064038003d4 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -24,7 +24,7 @@ #include -#include +#include #include #include #include @@ -150,7 +150,7 @@ static void logException(Context & context, QueryLogElement & elem) static void onExceptionBeforeStart(const String & query_for_logging, Context & context, time_t current_time) { /// Exception before the query execution. - context.getQuota().addError(); + context.getQuota()->used(Quota::ERRORS, 1, /* check_exceeded = */ false); const Settings & settings = context.getSettingsRef(); @@ -271,10 +271,9 @@ static std::tuple executeQueryImpl( /// Check the limits. checkASTSizeLimits(*ast, settings); - QuotaForIntervals & quota = context.getQuota(); - - quota.addQuery(); /// NOTE Seems that when new time interval has come, first query is not accounted in number of queries. - quota.checkExceeded(current_time); + auto quota = context.getQuota(); + quota->used(Quota::QUERIES, 1); + quota->checkExceeded(Quota::ERRORS); /// Put query to process list. But don't put SHOW PROCESSLIST query itself. ProcessList::EntryPtr process_list_entry; @@ -484,7 +483,7 @@ static std::tuple executeQueryImpl( auto exception_callback = [elem, &context, log_queries] () mutable { - context.getQuota().addError(); + context.getQuota()->used(Quota::ERRORS, 1, /* check_exceeded = */ false); elem.type = QueryLogElement::EXCEPTION_WHILE_PROCESSING; diff --git a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp index 5d632bdcef5..0522e7a5323 100644 --- a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp +++ b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.cpp @@ -198,7 +198,7 @@ void TreeExecutorBlockInputStream::setLimits(const IBlockInputStream::LocalLimit source->setLimits(limits_); } -void TreeExecutorBlockInputStream::setQuota(QuotaForIntervals & quota_) +void TreeExecutorBlockInputStream::setQuota(const std::shared_ptr & quota_) { for (auto & source : sources_with_progress) source->setQuota(quota_); diff --git a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h index da1d60dd972..176fbd06af8 100644 --- a/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h +++ b/dbms/src/Processors/Executors/TreeExecutorBlockInputStream.h @@ -31,7 +31,7 @@ public: void setProgressCallback(const ProgressCallback & callback) final; void setProcessListElement(QueryStatus * elem) final; void setLimits(const LocalLimits & limits_) final; - void setQuota(QuotaForIntervals & quota_) final; + void setQuota(const std::shared_ptr & quota_) final; void addTotalRowsApprox(size_t value) final; protected: diff --git a/dbms/src/Processors/Pipe.cpp b/dbms/src/Processors/Pipe.cpp index b31cfd58848..17b44a48ea1 100644 --- a/dbms/src/Processors/Pipe.cpp +++ b/dbms/src/Processors/Pipe.cpp @@ -97,7 +97,7 @@ void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) } } -void Pipe::setQuota(QuotaForIntervals & quota) +void Pipe::setQuota(const std::shared_ptr & quota) { for (auto & processor : processors) { diff --git a/dbms/src/Processors/Pipe.h b/dbms/src/Processors/Pipe.h index 72cb90c4b9e..d734c89f485 100644 --- a/dbms/src/Processors/Pipe.h +++ b/dbms/src/Processors/Pipe.h @@ -8,8 +8,6 @@ namespace DB class Pipe; using Pipes = std::vector; -class QuotaForIntervals; - /// Pipe is a set of processors which represents the part of pipeline with single output. /// All processors in pipe are connected. All ports are connected except the output one. class Pipe @@ -39,7 +37,7 @@ public: /// Specify quotas and limits for every ISourceWithProgress. void setLimits(const SourceWithProgress::LocalLimits & limits); - void setQuota(QuotaForIntervals & quota); + void setQuota(const std::shared_ptr & quota); /// Set information about preferred executor number for sources. void pinSources(size_t executor_number); diff --git a/dbms/src/Processors/Sources/SourceFromInputStream.h b/dbms/src/Processors/Sources/SourceFromInputStream.h index 888439f15d5..8e750a33faf 100644 --- a/dbms/src/Processors/Sources/SourceFromInputStream.h +++ b/dbms/src/Processors/Sources/SourceFromInputStream.h @@ -25,7 +25,7 @@ public: /// Implementation for methods from ISourceWithProgress. void setLimits(const LocalLimits & limits_) final { stream->setLimits(limits_); } - void setQuota(QuotaForIntervals & quota_) final { stream->setQuota(quota_); } + void setQuota(const std::shared_ptr & quota_) final { stream->setQuota(quota_); } void setProcessListElement(QueryStatus * elem) final { stream->setProcessListElement(elem); } void setProgressCallback(const ProgressCallback & callback) final { stream->setProgressCallback(callback); } void addTotalRowsApprox(size_t value) final { stream->addTotalRowsApprox(value); } diff --git a/dbms/src/Processors/Sources/SourceWithProgress.cpp b/dbms/src/Processors/Sources/SourceWithProgress.cpp index 21f9d5ca9bb..fac2a53ea54 100644 --- a/dbms/src/Processors/Sources/SourceWithProgress.cpp +++ b/dbms/src/Processors/Sources/SourceWithProgress.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include namespace DB { @@ -72,10 +72,8 @@ void SourceWithProgress::progress(const Progress & value) /// It is here for compatibility with IBlockInputsStream. limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); - if (quota != nullptr && limits.mode == LimitsMode::LIMITS_TOTAL) - { - quota->checkAndAddReadRowsBytes(time(nullptr), value.read_rows, value.read_bytes); - } + if (quota && limits.mode == LimitsMode::LIMITS_TOTAL) + quota->used({Quota::READ_ROWS, value.read_rows}, {Quota::READ_BYTES, value.read_bytes}); } } diff --git a/dbms/src/Processors/Sources/SourceWithProgress.h b/dbms/src/Processors/Sources/SourceWithProgress.h index 833e5eccb6f..59e8c6afa20 100644 --- a/dbms/src/Processors/Sources/SourceWithProgress.h +++ b/dbms/src/Processors/Sources/SourceWithProgress.h @@ -21,7 +21,7 @@ public: /// Set the quota. If you set a quota on the amount of raw data, /// then you should also set mode = LIMITS_TOTAL to LocalLimits with setLimits. - virtual void setQuota(QuotaForIntervals & quota_) = 0; + virtual void setQuota(const std::shared_ptr & quota_) = 0; /// Set the pointer to the process list item. /// General information about the resources spent on the request will be written into it. @@ -49,7 +49,7 @@ public: using LimitsMode = IBlockInputStream::LimitsMode; void setLimits(const LocalLimits & limits_) final { limits = limits_; } - void setQuota(QuotaForIntervals & quota_) final { quota = "a_; } + void setQuota(const std::shared_ptr & quota_) final { quota = quota_; } void setProcessListElement(QueryStatus * elem) final { process_list_elem = elem; } void setProgressCallback(const ProgressCallback & callback) final { progress_callback = callback; } void addTotalRowsApprox(size_t value) final { total_rows_approx += value; } @@ -60,7 +60,7 @@ protected: private: LocalLimits limits; - QuotaForIntervals * quota = nullptr; + std::shared_ptr quota; ProgressCallback progress_callback; QueryStatus * process_list_elem = nullptr; diff --git a/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp b/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp index 4947d11974b..1f621439048 100644 --- a/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp +++ b/dbms/src/Processors/Transforms/LimitsCheckingTransform.cpp @@ -1,5 +1,5 @@ #include -#include +#include namespace DB { @@ -73,7 +73,7 @@ void LimitsCheckingTransform::transform(Chunk & chunk) !limits.size_limits.check(info.rows, info.bytes, "result", ErrorCodes::TOO_MANY_ROWS_OR_BYTES)) stopReading(); - if (quota != nullptr) + if (quota) checkQuota(chunk); } } @@ -100,12 +100,8 @@ void LimitsCheckingTransform::checkQuota(Chunk & chunk) case LimitsMode::LIMITS_CURRENT: { - time_t current_time = time(nullptr); - double total_elapsed = info.total_stopwatch.elapsedSeconds(); - - quota->checkAndAddResultRowsBytes(current_time, chunk.getNumRows(), chunk.bytes()); - quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0)); - + UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds(); + quota->used({Quota::RESULT_ROWS, chunk.getNumRows()}, {Quota::RESULT_BYTES, chunk.bytes()}, {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); prev_elapsed = total_elapsed; break; } diff --git a/dbms/src/Processors/Transforms/LimitsCheckingTransform.h b/dbms/src/Processors/Transforms/LimitsCheckingTransform.h index 53116446a75..9410301030a 100644 --- a/dbms/src/Processors/Transforms/LimitsCheckingTransform.h +++ b/dbms/src/Processors/Transforms/LimitsCheckingTransform.h @@ -36,7 +36,7 @@ public: String getName() const override { return "LimitsCheckingTransform"; } - void setQuota(QuotaForIntervals & quota_) { quota = "a_; } + void setQuota(const std::shared_ptr & quota_) { quota = quota_; } protected: void transform(Chunk & chunk) override; @@ -44,8 +44,8 @@ protected: private: LocalLimits limits; - QuotaForIntervals * quota = nullptr; - double prev_elapsed = 0; + std::shared_ptr quota; + UInt64 prev_elapsed = 0; ProcessorProfileInfo info; From 55a05e5643429ca0a2cb75f874c2f268ef1dd465 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 4 Nov 2019 22:17:49 +0300 Subject: [PATCH 68/77] Add functions currentQuota() and currentQuotaKey(). --- dbms/src/Functions/currentQuota.cpp | 134 ++++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + 2 files changed, 136 insertions(+) create mode 100644 dbms/src/Functions/currentQuota.cpp diff --git a/dbms/src/Functions/currentQuota.cpp b/dbms/src/Functions/currentQuota.cpp new file mode 100644 index 00000000000..fef26f333fc --- /dev/null +++ b/dbms/src/Functions/currentQuota.cpp @@ -0,0 +1,134 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class FunctionCurrentQuota : public IFunction +{ + const String quota_name; + +public: + static constexpr auto name = "currentQuota"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getQuota()->getUsageInfo().quota_name); + } + + explicit FunctionCurrentQuota(const String & quota_name_) : quota_name{quota_name_} + { + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, quota_name); + } +}; + + +class FunctionCurrentQuotaId : public IFunction +{ + const UUID quota_id; + +public: + static constexpr auto name = "currentQuotaID"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getQuota()->getUsageInfo().quota_id); + } + + explicit FunctionCurrentQuotaId(const UUID quota_id_) : quota_id{quota_id_} + { + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeUUID().createColumnConst(input_rows_count, quota_id); + } +}; + + +class FunctionCurrentQuotaKey : public IFunction +{ + const String quota_key; + +public: + static constexpr auto name = "currentQuotaKey"; + static FunctionPtr create(const Context & context) + { + return std::make_shared(context.getQuota()->getUsageInfo().quota_key); + } + + explicit FunctionCurrentQuotaKey(const String & quota_key_) : quota_key{quota_key_} + { + } + + String getName() const override + { + return name; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + bool isDeterministic() const override { return false; } + + void executeImpl(Block & block, const ColumnNumbers &, size_t result, size_t input_rows_count) override + { + block.getByPosition(result).column = DataTypeString().createColumnConst(input_rows_count, quota_key); + } +}; + + +void registerFunctionCurrentQuota(FunctionFactory & factory) +{ + factory.registerFunction(); + factory.registerFunction(); + factory.registerFunction(); +} + +} diff --git a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp index ae75b9c0962..9529cd3a56a 100644 --- a/dbms/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/registerFunctionsMiscellaneous.cpp @@ -7,6 +7,7 @@ class FunctionFactory; void registerFunctionCurrentDatabase(FunctionFactory &); void registerFunctionCurrentUser(FunctionFactory &); +void registerFunctionCurrentQuota(FunctionFactory &); void registerFunctionHostName(FunctionFactory &); void registerFunctionFQDN(FunctionFactory &); void registerFunctionVisibleWidth(FunctionFactory &); @@ -62,6 +63,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) { registerFunctionCurrentDatabase(factory); registerFunctionCurrentUser(factory); + registerFunctionCurrentQuota(factory); registerFunctionHostName(factory); registerFunctionFQDN(factory); registerFunctionVisibleWidth(factory); From d9b1a733fcabb77315315894ece8a8b78bba668d Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 4 Nov 2019 22:18:17 +0300 Subject: [PATCH 69/77] Add table system.quotas. --- .../System/StorageSystemQuotaUsage.cpp | 79 +++++++++++ .../Storages/System/StorageSystemQuotaUsage.h | 28 ++++ .../Storages/System/StorageSystemQuotas.cpp | 124 ++++++++++++++++++ .../src/Storages/System/StorageSystemQuotas.h | 27 ++++ .../Storages/System/attachSystemTables.cpp | 4 + 5 files changed, 262 insertions(+) create mode 100644 dbms/src/Storages/System/StorageSystemQuotaUsage.cpp create mode 100644 dbms/src/Storages/System/StorageSystemQuotaUsage.h create mode 100644 dbms/src/Storages/System/StorageSystemQuotas.cpp create mode 100644 dbms/src/Storages/System/StorageSystemQuotas.h diff --git a/dbms/src/Storages/System/StorageSystemQuotaUsage.cpp b/dbms/src/Storages/System/StorageSystemQuotaUsage.cpp new file mode 100644 index 00000000000..8835e77eeb5 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -0,0 +1,79 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +NamesAndTypesList StorageSystemQuotaUsage::getNamesAndTypes() +{ + NamesAndTypesList names_and_types{ + {"name", std::make_shared()}, + {"id", std::make_shared()}, + {"key", std::make_shared()}, + {"duration", std::make_shared(std::make_shared())}, + {"end_of_interval", std::make_shared(std::make_shared())}}; + + for (auto resource_type : ext::range_with_static_cast(Quota::MAX_RESOURCE_TYPE)) + { + DataTypePtr data_type; + if (resource_type == Quota::EXECUTION_TIME) + data_type = std::make_shared(); + else + data_type = std::make_shared(); + + String column_name = Quota::resourceTypeToColumnName(resource_type); + names_and_types.push_back({column_name, std::make_shared(data_type)}); + names_and_types.push_back({String("max_") + column_name, std::make_shared(data_type)}); + } + return names_and_types; +} + + +void StorageSystemQuotaUsage::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const +{ + const auto & access_control = context.getAccessControlManager(); + for (const auto & info : access_control.getQuotaUsageInfo()) + { + for (const auto & interval : info.intervals) + { + size_t i = 0; + res_columns[i++]->insert(info.quota_name); + res_columns[i++]->insert(info.quota_id); + res_columns[i++]->insert(info.quota_key); + res_columns[i++]->insert(std::chrono::seconds{interval.duration}.count()); + res_columns[i++]->insert(std::chrono::system_clock::to_time_t(interval.end_of_interval)); + for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE)) + { + if (resource_type == Quota::EXECUTION_TIME) + { + res_columns[i++]->insert(Quota::executionTimeToSeconds(interval.used[resource_type])); + res_columns[i++]->insert(Quota::executionTimeToSeconds(interval.max[resource_type])); + } + else + { + res_columns[i++]->insert(interval.used[resource_type]); + res_columns[i++]->insert(interval.max[resource_type]); + } + } + } + + if (info.intervals.empty()) + { + size_t i = 0; + res_columns[i++]->insert(info.quota_name); + res_columns[i++]->insert(info.quota_id); + res_columns[i++]->insert(info.quota_key); + for (size_t j = 0; j != Quota::MAX_RESOURCE_TYPE * 2 + 2; ++j) + res_columns[i++]->insertDefault(); + } + } +} +} diff --git a/dbms/src/Storages/System/StorageSystemQuotaUsage.h b/dbms/src/Storages/System/StorageSystemQuotaUsage.h new file mode 100644 index 00000000000..f2151b27612 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemQuotaUsage.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +/** Implements the `quota_usage` system tables, which allows you to get information about + * how the quotas are used by all users. + */ +class StorageSystemQuotaUsage : public ext::shared_ptr_helper, public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemQuotaUsage"; } + static NamesAndTypesList getNamesAndTypes(); + +protected: + friend struct ext::shared_ptr_helper; + using IStorageSystemOneBlock::IStorageSystemOneBlock; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override; +}; + +} diff --git a/dbms/src/Storages/System/StorageSystemQuotas.cpp b/dbms/src/Storages/System/StorageSystemQuotas.cpp new file mode 100644 index 00000000000..b82e348c86d --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemQuotas.cpp @@ -0,0 +1,124 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + DataTypeEnum8::Values getKeyTypeEnumValues() + { + DataTypeEnum8::Values enum_values; + for (auto key_type : ext::range_with_static_cast(Quota::MAX_KEY_TYPE)) + enum_values.push_back({Quota::getNameOfKeyType(key_type), static_cast(key_type)}); + return enum_values; + } +} + + +NamesAndTypesList StorageSystemQuotas::getNamesAndTypes() +{ + NamesAndTypesList names_and_types{ + {"name", std::make_shared()}, + {"id", std::make_shared()}, + {"source", std::make_shared()}, + {"key_type", std::make_shared(getKeyTypeEnumValues())}, + {"roles", std::make_shared(std::make_shared())}, + {"all_roles", std::make_shared()}, + {"except_roles", std::make_shared(std::make_shared())}, + {"intervals.duration", std::make_shared(std::make_shared())}, + {"intervals.randomize_interval", std::make_shared(std::make_shared())}}; + + for (auto resource_type : ext::range_with_static_cast(Quota::MAX_RESOURCE_TYPE)) + { + DataTypePtr data_type; + if (resource_type == Quota::EXECUTION_TIME) + data_type = std::make_shared(); + else + data_type = std::make_shared(); + + String column_name = String("intervals.max_") + Quota::resourceTypeToColumnName(resource_type); + names_and_types.push_back({column_name, std::make_shared(data_type)}); + } + return names_and_types; +} + + +void StorageSystemQuotas::fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const +{ + size_t i = 0; + auto & name_column = *res_columns[i++]; + auto & id_column = *res_columns[i++]; + auto & storage_name_column = *res_columns[i++]; + auto & key_type_column = *res_columns[i++]; + auto & roles_data = assert_cast(*res_columns[i]).getData(); + auto & roles_offsets = assert_cast(*res_columns[i++]).getOffsets(); + auto & all_roles_column = *res_columns[i++]; + auto & except_roles_data = assert_cast(*res_columns[i]).getData(); + auto & except_roles_offsets = assert_cast(*res_columns[i++]).getOffsets(); + auto & durations_data = assert_cast(*res_columns[i]).getData(); + auto & durations_offsets = assert_cast(*res_columns[i++]).getOffsets(); + auto & randomize_intervals_data = assert_cast(*res_columns[i]).getData(); + auto & randomize_intervals_offsets = assert_cast(*res_columns[i++]).getOffsets(); + IColumn * limits_data[Quota::MAX_RESOURCE_TYPE]; + ColumnArray::Offsets * limits_offsets[Quota::MAX_RESOURCE_TYPE]; + for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE)) + { + limits_data[resource_type] = &assert_cast(*res_columns[i]).getData(); + limits_offsets[resource_type] = &assert_cast(*res_columns[i++]).getOffsets(); + } + + const auto & access_control = context.getAccessControlManager(); + for (const auto & id : access_control.findAll()) + { + auto quota = access_control.tryRead(id); + if (!quota) + continue; + const auto * storage = access_control.findStorage(id); + String storage_name = storage ? storage->getStorageName() : ""; + + name_column.insert(quota->getName()); + id_column.insert(id); + storage_name_column.insert(storage_name); + key_type_column.insert(static_cast(quota->key_type)); + + for (const auto & role : quota->roles) + roles_data.insert(role); + roles_offsets.push_back(roles_data.size()); + + all_roles_column.insert(static_cast(quota->all_roles)); + + for (const auto & except_role : quota->except_roles) + except_roles_data.insert(except_role); + except_roles_offsets.push_back(except_roles_data.size()); + + for (const auto & limits : quota->all_limits) + { + durations_data.insert(std::chrono::seconds{limits.duration}.count()); + randomize_intervals_data.insert(static_cast(limits.randomize_interval)); + for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE)) + { + if (resource_type == Quota::EXECUTION_TIME) + limits_data[resource_type]->insert(Quota::executionTimeToSeconds(limits.max[resource_type])); + else + limits_data[resource_type]->insert(limits.max[resource_type]); + } + } + + durations_offsets.push_back(durations_data.size()); + randomize_intervals_offsets.push_back(randomize_intervals_data.size()); + for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE)) + limits_offsets[resource_type]->push_back(limits_data[resource_type]->size()); + } +} +} diff --git a/dbms/src/Storages/System/StorageSystemQuotas.h b/dbms/src/Storages/System/StorageSystemQuotas.h new file mode 100644 index 00000000000..0f54f193654 --- /dev/null +++ b/dbms/src/Storages/System/StorageSystemQuotas.h @@ -0,0 +1,27 @@ +#pragma once + +#include +#include + + +namespace DB +{ + +class Context; + + +/** Implements the `quotas` system tables, which allows you to get information about quotas. + */ +class StorageSystemQuotas : public ext::shared_ptr_helper, public IStorageSystemOneBlock +{ +public: + std::string getName() const override { return "SystemQuotas"; } + static NamesAndTypesList getNamesAndTypes(); + +protected: + friend struct ext::shared_ptr_helper; + using IStorageSystemOneBlock::IStorageSystemOneBlock; + void fillData(MutableColumns & res_columns, const Context & context, const SelectQueryInfo &) const override; +}; + +} diff --git a/dbms/src/Storages/System/attachSystemTables.cpp b/dbms/src/Storages/System/attachSystemTables.cpp index 528bdd06a21..2b8e630cbed 100644 --- a/dbms/src/Storages/System/attachSystemTables.cpp +++ b/dbms/src/Storages/System/attachSystemTables.cpp @@ -25,6 +25,8 @@ #include #include #include +#include +#include #include #include #include @@ -52,6 +54,8 @@ void attachSystemTablesLocal(IDatabase & system_database) system_database.attachTable("functions", StorageSystemFunctions::create("functions")); system_database.attachTable("events", StorageSystemEvents::create("events")); system_database.attachTable("settings", StorageSystemSettings::create("settings")); + system_database.attachTable("quotas", StorageSystemQuotas::create("quotas")); + system_database.attachTable("quota_usage", StorageSystemQuotaUsage::create("quota_usage")); system_database.attachTable("merge_tree_settings", SystemMergeTreeSettings::create("merge_tree_settings")); system_database.attachTable("build_options", StorageSystemBuildOptions::create("build_options")); system_database.attachTable("formats", StorageSystemFormats::create("formats")); From 38fa1af5cadb7188ea1a0b570248434049fc024f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 11 Nov 2019 04:11:32 +0300 Subject: [PATCH 70/77] Disable quota consumptions for the system queries and for selecting from the `system.quota` and `system.quotas` tables. --- dbms/src/Interpreters/IInterpreter.h | 3 ++ .../Interpreters/InterpreterSelectQuery.cpp | 15 ++++++- .../src/Interpreters/InterpreterSelectQuery.h | 5 ++- .../InterpreterSelectWithUnionQuery.cpp | 13 +++++++ .../InterpreterSelectWithUnionQuery.h | 5 ++- .../Interpreters/InterpreterSystemQuery.cpp | 4 +- .../src/Interpreters/InterpreterSystemQuery.h | 3 ++ dbms/src/Interpreters/SelectQueryOptions.h | 19 ++++----- dbms/src/Interpreters/executeQuery.cpp | 39 +++++++++++-------- 9 files changed, 74 insertions(+), 32 deletions(-) diff --git a/dbms/src/Interpreters/IInterpreter.h b/dbms/src/Interpreters/IInterpreter.h index e1090061cf3..e2248a7ec7a 100644 --- a/dbms/src/Interpreters/IInterpreter.h +++ b/dbms/src/Interpreters/IInterpreter.h @@ -22,6 +22,9 @@ public: virtual bool canExecuteWithProcessors() const { return false; } + virtual bool ignoreQuota() const { return false; } + virtual bool ignoreLimits() const { return false; } + virtual ~IInterpreter() {} }; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 7134b52d7cd..d4565f6894d 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -419,6 +419,17 @@ InterpreterSelectQuery::InterpreterSelectQuery( /// null non-const columns to avoid useless memory allocations. However, a valid block sample /// requires all columns to be of size 0, thus we need to sanitize the block here. sanitizeBlock(result_header); + + /// Remove limits for some tables in the `system` database. + if (storage && (storage->getDatabaseName() == "system")) + { + String table_name = storage->getTableName(); + if ((table_name == "quotas") || (table_name == "quota_usage") || (table_name == "one")) + { + options.ignore_quota = true; + options.ignore_limits = true; + } + } } @@ -1783,7 +1794,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (!options.ignore_limits) stream->setLimits(limits); - if (options.to_stage == QueryProcessingStage::Complete) + if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) stream->setQuota(quota); } @@ -1793,7 +1804,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (!options.ignore_limits) pipe.setLimits(limits); - if (options.to_stage == QueryProcessingStage::Complete) + if (!options.ignore_quota && (options.to_stage == QueryProcessingStage::Complete)) pipe.setQuota(quota); } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 083a4ebe680..6b95d7aeea7 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -74,6 +74,9 @@ public: QueryPipeline executeWithProcessors() override; bool canExecuteWithProcessors() const override { return true; } + bool ignoreLimits() const override { return options.ignore_limits; } + bool ignoreQuota() const override { return options.ignore_quota; } + Block getSampleBlock(); void ignoreWithTotals(); @@ -260,7 +263,7 @@ private: */ void initSettings(); - const SelectQueryOptions options; + SelectQueryOptions options; ASTPtr query_ptr; std::shared_ptr context; SyntaxAnalyzerResultPtr syntax_analyzer_result; diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 2d7fde72875..e76f2668d3d 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -107,6 +107,19 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery( result_header = getCommonHeaderForUnion(headers); } + + /// InterpreterSelectWithUnionQuery ignores limits if all nested interpreters ignore limits. + bool all_nested_ignore_limits = true; + bool all_nested_ignore_quota = true; + for (auto & interpreter : nested_interpreters) + { + if (!interpreter->ignoreLimits()) + all_nested_ignore_limits = false; + if (!interpreter->ignoreQuota()) + all_nested_ignore_quota = false; + } + options.ignore_limits |= all_nested_ignore_limits; + options.ignore_quota |= all_nested_ignore_quota; } diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h index 4e425d260e6..e18627fec2a 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.h @@ -34,6 +34,9 @@ public: QueryPipeline executeWithProcessors() override; bool canExecuteWithProcessors() const override { return true; } + bool ignoreLimits() const override { return options.ignore_limits; } + bool ignoreQuota() const override { return options.ignore_quota; } + Block getSampleBlock(); static Block getSampleBlock( @@ -45,7 +48,7 @@ public: ASTPtr getQuery() const { return query_ptr; } private: - const SelectQueryOptions options; + SelectQueryOptions options; ASTPtr query_ptr; std::shared_ptr context; diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.cpp b/dbms/src/Interpreters/InterpreterSystemQuery.cpp index 664efca90f9..c742ac37a5f 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSystemQuery.cpp @@ -121,7 +121,9 @@ void startStopAction(Context & context, ASTSystemQuery & query, StorageActionBlo InterpreterSystemQuery::InterpreterSystemQuery(const ASTPtr & query_ptr_, Context & context_) - : query_ptr(query_ptr_->clone()), context(context_), log(&Poco::Logger::get("InterpreterSystemQuery")) {} + : query_ptr(query_ptr_->clone()), context(context_), log(&Poco::Logger::get("InterpreterSystemQuery")) +{ +} BlockIO InterpreterSystemQuery::execute() diff --git a/dbms/src/Interpreters/InterpreterSystemQuery.h b/dbms/src/Interpreters/InterpreterSystemQuery.h index 31945745c1e..97ff9d348e6 100644 --- a/dbms/src/Interpreters/InterpreterSystemQuery.h +++ b/dbms/src/Interpreters/InterpreterSystemQuery.h @@ -20,6 +20,9 @@ public: BlockIO execute() override; + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + private: ASTPtr query_ptr; Context & context; diff --git a/dbms/src/Interpreters/SelectQueryOptions.h b/dbms/src/Interpreters/SelectQueryOptions.h index 4fd94a830b8..a49245f5609 100644 --- a/dbms/src/Interpreters/SelectQueryOptions.h +++ b/dbms/src/Interpreters/SelectQueryOptions.h @@ -24,19 +24,16 @@ struct SelectQueryOptions { QueryProcessingStage::Enum to_stage; size_t subquery_depth; - bool only_analyze; - bool modify_inplace; - bool remove_duplicates; - bool ignore_limits; + bool only_analyze = false; + bool modify_inplace = false; + bool remove_duplicates = false; + bool ignore_quota = false; + bool ignore_limits = false; SelectQueryOptions(QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, size_t depth = 0) - : to_stage(stage) - , subquery_depth(depth) - , only_analyze(false) - , modify_inplace(false) - , remove_duplicates(false) - , ignore_limits(false) - {} + : to_stage(stage), subquery_depth(depth) + { + } SelectQueryOptions copy() const { return *this; } diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 064038003d4..2c6bf087f8d 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -271,10 +271,6 @@ static std::tuple executeQueryImpl( /// Check the limits. checkASTSizeLimits(*ast, settings); - auto quota = context.getQuota(); - quota->used(Quota::QUERIES, 1); - quota->checkExceeded(Quota::ERRORS); - /// Put query to process list. But don't put SHOW PROCESSLIST query itself. ProcessList::EntryPtr process_list_entry; if (!internal && !ast->as()) @@ -312,6 +308,21 @@ static std::tuple executeQueryImpl( auto interpreter = InterpreterFactory::get(ast, context, stage); bool use_processors = settings.experimental_use_processors && allow_processors && interpreter->canExecuteWithProcessors(); + QuotaContextPtr quota; + if (!interpreter->ignoreQuota()) + { + quota = context.getQuota(); + quota->used(Quota::QUERIES, 1); + quota->checkExceeded(Quota::ERRORS); + } + + IBlockInputStream::LocalLimits limits; + if (!interpreter->ignoreLimits()) + { + limits.mode = IBlockInputStream::LIMITS_CURRENT; + limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); + } + if (use_processors) pipeline = interpreter->executeWithProcessors(); else @@ -338,17 +349,12 @@ static std::tuple executeQueryImpl( /// Hold element of process list till end of query execution. res.process_list_entry = process_list_entry; - IBlockInputStream::LocalLimits limits; - limits.mode = IBlockInputStream::LIMITS_CURRENT; - limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode); - if (use_processors) { - pipeline.setProgressCallback(context.getProgressCallback()); - pipeline.setProcessListElement(context.getProcessListElement()); - /// Limits on the result, the quota on the result, and also callback for progress. /// Limits apply only to the final result. + pipeline.setProgressCallback(context.getProgressCallback()); + pipeline.setProcessListElement(context.getProcessListElement()); if (stage == QueryProcessingStage::Complete) { pipeline.resize(1); @@ -362,17 +368,18 @@ static std::tuple executeQueryImpl( } else { + /// Limits on the result, the quota on the result, and also callback for progress. + /// Limits apply only to the final result. if (res.in) { res.in->setProgressCallback(context.getProgressCallback()); res.in->setProcessListElement(context.getProcessListElement()); - - /// Limits on the result, the quota on the result, and also callback for progress. - /// Limits apply only to the final result. if (stage == QueryProcessingStage::Complete) { - res.in->setLimits(limits); - res.in->setQuota(quota); + if (!interpreter->ignoreQuota()) + res.in->setQuota(quota); + if (!interpreter->ignoreLimits()) + res.in->setLimits(limits); } } From ae296287793dc321c5ab11c6a8e42a4a49f1be4f Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 16 Nov 2019 16:54:41 +0300 Subject: [PATCH 71/77] Add an integration test for quotas. --- dbms/tests/integration/test_quota/__init__.py | 0 .../test_quota/configs/users.d/quota.xml | 1 + .../integration/test_quota/configs/users.xml | 17 ++ .../integration/test_quota/no_quotas.xml | 3 + .../integration/test_quota/normal_limits.xml | 17 ++ .../integration/test_quota/simpliest.xml | 7 + dbms/tests/integration/test_quota/test.py | 177 ++++++++++++++++++ .../integration/test_quota/tiny_limits.xml | 17 ++ .../tests/integration/test_quota/tracking.xml | 17 ++ .../integration/test_quota/two_intervals.xml | 20 ++ .../integration/test_quota/two_quotas.xml | 29 +++ 11 files changed, 305 insertions(+) create mode 100644 dbms/tests/integration/test_quota/__init__.py create mode 120000 dbms/tests/integration/test_quota/configs/users.d/quota.xml create mode 100644 dbms/tests/integration/test_quota/configs/users.xml create mode 100644 dbms/tests/integration/test_quota/no_quotas.xml create mode 100644 dbms/tests/integration/test_quota/normal_limits.xml create mode 100644 dbms/tests/integration/test_quota/simpliest.xml create mode 100644 dbms/tests/integration/test_quota/test.py create mode 100644 dbms/tests/integration/test_quota/tiny_limits.xml create mode 100644 dbms/tests/integration/test_quota/tracking.xml create mode 100644 dbms/tests/integration/test_quota/two_intervals.xml create mode 100644 dbms/tests/integration/test_quota/two_quotas.xml diff --git a/dbms/tests/integration/test_quota/__init__.py b/dbms/tests/integration/test_quota/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_quota/configs/users.d/quota.xml b/dbms/tests/integration/test_quota/configs/users.d/quota.xml new file mode 120000 index 00000000000..9b12dec9c53 --- /dev/null +++ b/dbms/tests/integration/test_quota/configs/users.d/quota.xml @@ -0,0 +1 @@ +../../normal_limits.xml \ No newline at end of file diff --git a/dbms/tests/integration/test_quota/configs/users.xml b/dbms/tests/integration/test_quota/configs/users.xml new file mode 100644 index 00000000000..4412345a731 --- /dev/null +++ b/dbms/tests/integration/test_quota/configs/users.xml @@ -0,0 +1,17 @@ + + + + + + + + + + + ::/0 + + default + myQuota + + + diff --git a/dbms/tests/integration/test_quota/no_quotas.xml b/dbms/tests/integration/test_quota/no_quotas.xml new file mode 100644 index 00000000000..9aba4ac0914 --- /dev/null +++ b/dbms/tests/integration/test_quota/no_quotas.xml @@ -0,0 +1,3 @@ + + + diff --git a/dbms/tests/integration/test_quota/normal_limits.xml b/dbms/tests/integration/test_quota/normal_limits.xml new file mode 100644 index 00000000000..b7c3a67b5cc --- /dev/null +++ b/dbms/tests/integration/test_quota/normal_limits.xml @@ -0,0 +1,17 @@ + + + + + + + 31556952 + + + 1000 + 0 + 1000 + 0 + + + + diff --git a/dbms/tests/integration/test_quota/simpliest.xml b/dbms/tests/integration/test_quota/simpliest.xml new file mode 100644 index 00000000000..6d51d68d8d9 --- /dev/null +++ b/dbms/tests/integration/test_quota/simpliest.xml @@ -0,0 +1,7 @@ + + + + + + + diff --git a/dbms/tests/integration/test_quota/test.py b/dbms/tests/integration/test_quota/test.py new file mode 100644 index 00000000000..5dae1be5e57 --- /dev/null +++ b/dbms/tests/integration/test_quota/test.py @@ -0,0 +1,177 @@ +import pytest +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry +import os +import re +import time + +cluster = ClickHouseCluster(__file__) +instance = cluster.add_instance('instance', + config_dir="configs") + +query_from_system_quotas = "SELECT * FROM system.quotas ORDER BY name"; + +query_from_system_quota_usage = "SELECT id, key, duration, "\ + "queries, errors, result_rows, result_bytes, read_rows, read_bytes "\ + "FROM system.quota_usage ORDER BY id, key, duration"; + +def system_quotas(): + return instance.query(query_from_system_quotas).rstrip('\n') + +def system_quota_usage(): + return instance.query(query_from_system_quota_usage).rstrip('\n') + + +def copy_quota_xml(local_file_name, reload_immediately = True): + script_dir = os.path.dirname(os.path.realpath(__file__)) + instance.copy_file_to_container(os.path.join(script_dir, local_file_name), '/etc/clickhouse-server/users.d/quota.xml') + if reload_immediately: + instance.query("SYSTEM RELOAD CONFIG") + + +@pytest.fixture(scope="module", autouse=True) +def started_cluster(): + try: + cluster.start() + + instance.query("CREATE TABLE test_table(x UInt32) ENGINE = MergeTree ORDER BY tuple()") + instance.query("INSERT INTO test_table SELECT number FROM numbers(50)") + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(autouse=True) +def reset_quotas_and_usage_info(): + try: + yield + finally: + copy_quota_xml('simpliest.xml') # To reset usage info. + copy_quota_xml('normal_limits.xml') + + +def test_quota_from_users_xml(): + assert instance.query("SELECT currentQuota()") == "myQuota\n" + assert instance.query("SELECT currentQuotaID()") == "e651da9c-a748-8703-061a-7e5e5096dae7\n" + assert instance.query("SELECT currentQuotaKey()") == "default\n" + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0" + + instance.query("SELECT * from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t1\t0\t50\t200\t50\t200" + + instance.query("SELECT COUNT() from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t2\t0\t51\t208\t50\t200" + + +def test_simpliest_quota(): + # Simpliest quota doesn't even track usage. + copy_quota_xml('simpliest.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[]\t[]\t[]\t[]\t[]\t[]\t[]\t[]\t[]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N" + + instance.query("SELECT * from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N" + + +def test_tracking_quota(): + # Now we're tracking usage. + copy_quota_xml('tracking.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[0]\t[0]\t[0]\t[0]\t[0]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0" + + instance.query("SELECT * from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t1\t0\t50\t200\t50\t200" + + instance.query("SELECT COUNT() from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t2\t0\t51\t208\t50\t200" + + +def test_exceed_quota(): + # Change quota, now the limits are tiny so we will exceed the quota. + copy_quota_xml('tiny_limits.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1]\t[1]\t[1]\t[0]\t[1]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0" + + assert re.search("Quota.*has\ been\ exceeded", instance.query_and_get_error("SELECT * from test_table")) + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t1\t1\t0\t0\t50\t0" + + # Change quota, now the limits are enough to execute queries. + copy_quota_xml('normal_limits.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t1\t1\t0\t0\t50\t0" + + instance.query("SELECT * from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t2\t1\t50\t200\t100\t200" + + +def test_add_remove_interval(): + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0" + + # Add interval. + copy_quota_xml('two_intervals.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952,63113904]\t[0,1]\t[1000,0]\t[0,0]\t[0,0]\t[0,30000]\t[1000,0]\t[0,20000]\t[0,120]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0\n"\ + "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t63113904\t0\t0\t0\t0\t0\t0" + + instance.query("SELECT * from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t1\t0\t50\t200\t50\t200\n"\ + "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t63113904\t1\t0\t50\t200\t50\t200" + + # Remove interval. + copy_quota_xml('normal_limits.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t1\t0\t50\t200\t50\t200" + + instance.query("SELECT * from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t2\t0\t100\t400\t100\t400" + + # Remove all intervals. + copy_quota_xml('simpliest.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[]\t[]\t[]\t[]\t[]\t[]\t[]\t[]\t[]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N" + + instance.query("SELECT * from test_table") + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N\t\\N" + + # Add one interval back. + copy_quota_xml('normal_limits.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0" + + +def test_add_remove_quota(): + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0" + + # Add quota. + copy_quota_xml('two_quotas.xml') + assert system_quotas() ==\ + "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]\n"\ + "myQuota2\t4590510c-4d13-bf21-ec8a-c2187b092e73\tusers.xml\tclient key or user name\t[]\t0\t[]\t[3600,2629746]\t[1,0]\t[0,0]\t[0,0]\t[4000,0]\t[400000,0]\t[4000,0]\t[400000,0]\t[60,1800]" + + # Drop quota. + copy_quota_xml('normal_limits.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + + # Drop all quotas. + copy_quota_xml('no_quotas.xml') + assert system_quotas() == "" + assert system_quota_usage() == "" + + # Add one quota back. + copy_quota_xml('normal_limits.xml') + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + assert system_quota_usage() == "e651da9c-a748-8703-061a-7e5e5096dae7\tdefault\t31556952\t0\t0\t0\t0\t0\t0" + + +def test_reload_users_xml_by_timer(): + assert system_quotas() == "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1000]\t[0]\t[0]\t[0]\t[1000]\t[0]\t[0]" + + time.sleep(1) # The modification time of the 'quota.xml' file should be different, + # because config files are reload by timer only when the modification time is changed. + copy_quota_xml('tiny_limits.xml', reload_immediately=False) + assert_eq_with_retry(instance, query_from_system_quotas, "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1]\t[1]\t[1]\t[0]\t[1]\t[0]\t[0]") diff --git a/dbms/tests/integration/test_quota/tiny_limits.xml b/dbms/tests/integration/test_quota/tiny_limits.xml new file mode 100644 index 00000000000..3ab8858738a --- /dev/null +++ b/dbms/tests/integration/test_quota/tiny_limits.xml @@ -0,0 +1,17 @@ + + + + + + + 31556952 + + + 1 + 1 + 1 + 1 + + + + diff --git a/dbms/tests/integration/test_quota/tracking.xml b/dbms/tests/integration/test_quota/tracking.xml new file mode 100644 index 00000000000..47e12bf8005 --- /dev/null +++ b/dbms/tests/integration/test_quota/tracking.xml @@ -0,0 +1,17 @@ + + + + + + + 31556952 + + + 0 + 0 + 0 + 0 + + + + diff --git a/dbms/tests/integration/test_quota/two_intervals.xml b/dbms/tests/integration/test_quota/two_intervals.xml new file mode 100644 index 00000000000..d0de605b895 --- /dev/null +++ b/dbms/tests/integration/test_quota/two_intervals.xml @@ -0,0 +1,20 @@ + + + + + + 31556952 + 1000 + 1000 + + + + true + 63113904 + 20000 + 30000 + 120 + + + + diff --git a/dbms/tests/integration/test_quota/two_quotas.xml b/dbms/tests/integration/test_quota/two_quotas.xml new file mode 100644 index 00000000000..c08cc82aca7 --- /dev/null +++ b/dbms/tests/integration/test_quota/two_quotas.xml @@ -0,0 +1,29 @@ + + + + + + 31556952 + 1000 + 1000 + + + + + + + true + 3600 + 4000 + 4000 + 400000 + 400000 + 60 + + + 2629746 + 1800 + + + + From 3e6619c0762ad039026834f4a3a5ea2b55a7caea Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 16 Nov 2019 16:54:52 +0300 Subject: [PATCH 72/77] Move IntervalKind-related functions to a separate file. --- dbms/src/Common/IntervalKind.cpp | 162 ++++++++++++++++++ dbms/src/Common/IntervalKind.h | 54 ++++++ dbms/src/DataTypes/DataTypeInterval.cpp | 16 +- dbms/src/DataTypes/DataTypeInterval.h | 39 +---- dbms/src/Functions/FunctionBinaryArithmetic.h | 2 +- dbms/src/Functions/FunctionsConversion.h | 4 +- dbms/src/Functions/toStartOfInterval.cpp | 60 +++---- dbms/src/Parsers/CommonParsers.h | 93 ---------- dbms/src/Parsers/ExpressionElementParsers.cpp | 90 ++-------- dbms/src/Parsers/ExpressionListParsers.cpp | 11 +- dbms/src/Parsers/parseIntervalKind.cpp | 68 ++++++++ dbms/src/Parsers/parseIntervalKind.h | 11 ++ 12 files changed, 357 insertions(+), 253 deletions(-) create mode 100644 dbms/src/Common/IntervalKind.cpp create mode 100644 dbms/src/Common/IntervalKind.h create mode 100644 dbms/src/Parsers/parseIntervalKind.cpp create mode 100644 dbms/src/Parsers/parseIntervalKind.h diff --git a/dbms/src/Common/IntervalKind.cpp b/dbms/src/Common/IntervalKind.cpp new file mode 100644 index 00000000000..9443844a54b --- /dev/null +++ b/dbms/src/Common/IntervalKind.cpp @@ -0,0 +1,162 @@ +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + +const char * IntervalKind::toString() const +{ + switch (kind) + { + case IntervalKind::Second: return "Second"; + case IntervalKind::Minute: return "Minute"; + case IntervalKind::Hour: return "Hour"; + case IntervalKind::Day: return "Day"; + case IntervalKind::Week: return "Week"; + case IntervalKind::Month: return "Month"; + case IntervalKind::Quarter: return "Quarter"; + case IntervalKind::Year: return "Year"; + } + __builtin_unreachable(); +} + + +Int32 IntervalKind::toAvgSeconds() const +{ + switch (kind) + { + case IntervalKind::Second: return 1; + case IntervalKind::Minute: return 60; + case IntervalKind::Hour: return 3600; + case IntervalKind::Day: return 86400; + case IntervalKind::Week: return 604800; + case IntervalKind::Month: return 2629746; /// Exactly 1/12 of a year. + case IntervalKind::Quarter: return 7889238; /// Exactly 1/4 of a year. + case IntervalKind::Year: return 31556952; /// The average length of a Gregorian year is equal to 365.2425 days + } + __builtin_unreachable(); +} + + +IntervalKind IntervalKind::fromAvgSeconds(Int64 num_seconds) +{ + if (num_seconds) + { + if (!(num_seconds % 31556952)) + return IntervalKind::Year; + if (!(num_seconds % 7889238)) + return IntervalKind::Quarter; + if (!(num_seconds % 604800)) + return IntervalKind::Week; + if (!(num_seconds % 2629746)) + return IntervalKind::Month; + if (!(num_seconds % 86400)) + return IntervalKind::Day; + if (!(num_seconds % 3600)) + return IntervalKind::Hour; + if (!(num_seconds % 60)) + return IntervalKind::Minute; + } + return IntervalKind::Second; +} + + +const char * IntervalKind::toKeyword() const +{ + switch (kind) + { + case IntervalKind::Second: return "SECOND"; + case IntervalKind::Minute: return "MINUTE"; + case IntervalKind::Hour: return "HOUR"; + case IntervalKind::Day: return "DAY"; + case IntervalKind::Week: return "WEEK"; + case IntervalKind::Month: return "MONTH"; + case IntervalKind::Quarter: return "QUARTER"; + case IntervalKind::Year: return "YEAR"; + } + __builtin_unreachable(); +} + + +const char * IntervalKind::toDateDiffUnit() const +{ + switch (kind) + { + case IntervalKind::Second: + return "second"; + case IntervalKind::Minute: + return "minute"; + case IntervalKind::Hour: + return "hour"; + case IntervalKind::Day: + return "day"; + case IntervalKind::Week: + return "week"; + case IntervalKind::Month: + return "month"; + case IntervalKind::Quarter: + return "quarter"; + case IntervalKind::Year: + return "year"; + } + __builtin_unreachable(); +} + + +const char * IntervalKind::toNameOfFunctionToIntervalDataType() const +{ + switch (kind) + { + case IntervalKind::Second: + return "toIntervalSecond"; + case IntervalKind::Minute: + return "toIntervalMinute"; + case IntervalKind::Hour: + return "toIntervalHour"; + case IntervalKind::Day: + return "toIntervalDay"; + case IntervalKind::Week: + return "toIntervalWeek"; + case IntervalKind::Month: + return "toIntervalMonth"; + case IntervalKind::Quarter: + return "toIntervalQuarter"; + case IntervalKind::Year: + return "toIntervalYear"; + } + __builtin_unreachable(); +} + + +const char * IntervalKind::toNameOfFunctionExtractTimePart() const +{ + switch (kind) + { + case IntervalKind::Second: + return "toSecond"; + case IntervalKind::Minute: + return "toMinute"; + case IntervalKind::Hour: + return "toHour"; + case IntervalKind::Day: + return "toDayOfMonth"; + case IntervalKind::Week: + // TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15'))) + // else if (ParserKeyword("WEEK").ignore(pos, expected)) + // function_name = "toRelativeWeekNum"; + throw Exception("The syntax 'EXTRACT(WEEK FROM date)' is not supported, cannot extract the number of a week", ErrorCodes::SYNTAX_ERROR); + case IntervalKind::Month: + return "toMonth"; + case IntervalKind::Quarter: + return "toQuarter"; + case IntervalKind::Year: + return "toYear"; + } + __builtin_unreachable(); +} +} diff --git a/dbms/src/Common/IntervalKind.h b/dbms/src/Common/IntervalKind.h new file mode 100644 index 00000000000..9b7c4bd504e --- /dev/null +++ b/dbms/src/Common/IntervalKind.h @@ -0,0 +1,54 @@ +#pragma once + +#include + + +namespace DB +{ +/// Kind of a temporal interval. +struct IntervalKind +{ + enum Kind + { + Second, + Minute, + Hour, + Day, + Week, + Month, + Quarter, + Year, + }; + Kind kind = Second; + + IntervalKind(Kind kind_ = Second) : kind(kind_) {} + operator Kind() const { return kind; } + + const char * toString() const; + + /// Returns number of seconds in one interval. + /// For `Month`, `Quarter` and `Year` the function returns an average number of seconds. + Int32 toAvgSeconds() const; + + /// Chooses an interval kind based on number of seconds. + /// For example, `IntervalKind::fromAvgSeconds(3600)` returns `IntervalKind::Hour`. + static IntervalKind fromAvgSeconds(Int64 num_seconds); + + /// Returns an uppercased version of what `toString()` returns. + const char * toKeyword() const; + + /// Returns the string which can be passed to the `unit` parameter of the dateDiff() function. + /// For example, `IntervalKind{IntervalKind::Day}.getDateDiffParameter()` returns "day". + const char * toDateDiffUnit() const; + + /// Returns the name of the function converting a number to the interval data type. + /// For example, `IntervalKind{IntervalKind::Day}.getToIntervalDataTypeFunctionName()` + /// returns "toIntervalDay". + const char * toNameOfFunctionToIntervalDataType() const; + + /// Returns the name of the function extracting time part from a date or a time. + /// For example, `IntervalKind{IntervalKind::Day}.getExtractTimePartFunctionName()` + /// returns "toDayOfMonth". + const char * toNameOfFunctionExtractTimePart() const; +}; +} diff --git a/dbms/src/DataTypes/DataTypeInterval.cpp b/dbms/src/DataTypes/DataTypeInterval.cpp index c7ee3ede334..57d071a8666 100644 --- a/dbms/src/DataTypes/DataTypeInterval.cpp +++ b/dbms/src/DataTypes/DataTypeInterval.cpp @@ -13,14 +13,14 @@ bool DataTypeInterval::equals(const IDataType & rhs) const void registerDataTypeInterval(DataTypeFactory & factory) { - factory.registerSimpleDataType("IntervalSecond", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Second)); }); - factory.registerSimpleDataType("IntervalMinute", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Minute)); }); - factory.registerSimpleDataType("IntervalHour", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Hour)); }); - factory.registerSimpleDataType("IntervalDay", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Day)); }); - factory.registerSimpleDataType("IntervalWeek", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Week)); }); - factory.registerSimpleDataType("IntervalMonth", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Month)); }); - factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Quarter)); }); - factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(DataTypeInterval::Year)); }); + factory.registerSimpleDataType("IntervalSecond", [] { return DataTypePtr(std::make_shared(IntervalKind::Second)); }); + factory.registerSimpleDataType("IntervalMinute", [] { return DataTypePtr(std::make_shared(IntervalKind::Minute)); }); + factory.registerSimpleDataType("IntervalHour", [] { return DataTypePtr(std::make_shared(IntervalKind::Hour)); }); + factory.registerSimpleDataType("IntervalDay", [] { return DataTypePtr(std::make_shared(IntervalKind::Day)); }); + factory.registerSimpleDataType("IntervalWeek", [] { return DataTypePtr(std::make_shared(IntervalKind::Week)); }); + factory.registerSimpleDataType("IntervalMonth", [] { return DataTypePtr(std::make_shared(IntervalKind::Month)); }); + factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared(IntervalKind::Quarter)); }); + factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared(IntervalKind::Year)); }); } } diff --git a/dbms/src/DataTypes/DataTypeInterval.h b/dbms/src/DataTypes/DataTypeInterval.h index fa99ac430b6..111a2489d65 100644 --- a/dbms/src/DataTypes/DataTypeInterval.h +++ b/dbms/src/DataTypes/DataTypeInterval.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -16,47 +17,17 @@ namespace DB */ class DataTypeInterval final : public DataTypeNumberBase { -public: - enum Kind - { - Second, - Minute, - Hour, - Day, - Week, - Month, - Quarter, - Year - }; - private: - Kind kind; + IntervalKind kind; public: static constexpr bool is_parametric = true; - Kind getKind() const { return kind; } + IntervalKind getKind() const { return kind; } - const char * kindToString() const - { - switch (kind) - { - case Second: return "Second"; - case Minute: return "Minute"; - case Hour: return "Hour"; - case Day: return "Day"; - case Week: return "Week"; - case Month: return "Month"; - case Quarter: return "Quarter"; - case Year: return "Year"; - } + DataTypeInterval(IntervalKind kind_) : kind(kind_) {} - __builtin_unreachable(); - } - - DataTypeInterval(Kind kind_) : kind(kind_) {} - - std::string doGetName() const override { return std::string("Interval") + kindToString(); } + std::string doGetName() const override { return std::string("Interval") + kind.toString(); } const char * getFamilyName() const override { return "Interval"; } TypeIndex getTypeId() const override { return TypeIndex::Interval; } diff --git a/dbms/src/Functions/FunctionBinaryArithmetic.h b/dbms/src/Functions/FunctionBinaryArithmetic.h index 407c096a9e5..cab13a405c7 100644 --- a/dbms/src/Functions/FunctionBinaryArithmetic.h +++ b/dbms/src/Functions/FunctionBinaryArithmetic.h @@ -508,7 +508,7 @@ class FunctionBinaryArithmetic : public IFunction } std::stringstream function_name; - function_name << (function_is_plus ? "add" : "subtract") << interval_data_type->kindToString() << 's'; + function_name << (function_is_plus ? "add" : "subtract") << interval_data_type->getKind().toString() << 's'; return FunctionFactory::instance().get(function_name.str(), context); } diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index 09a23f83414..e0f828c395a 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -735,7 +735,7 @@ struct NameToDecimal128 { static constexpr auto name = "toDecimal128"; }; struct NameToInterval ## INTERVAL_KIND \ { \ static constexpr auto name = "toInterval" #INTERVAL_KIND; \ - static constexpr int kind = DataTypeInterval::INTERVAL_KIND; \ + static constexpr auto kind = IntervalKind::INTERVAL_KIND; \ }; DEFINE_NAME_TO_INTERVAL(Second) @@ -786,7 +786,7 @@ public: if constexpr (std::is_same_v) { - return std::make_shared(DataTypeInterval::Kind(Name::kind)); + return std::make_shared(Name::kind); } else if constexpr (to_decimal) { diff --git a/dbms/src/Functions/toStartOfInterval.cpp b/dbms/src/Functions/toStartOfInterval.cpp index 21e500602e1..29ec814a6ee 100644 --- a/dbms/src/Functions/toStartOfInterval.cpp +++ b/dbms/src/Functions/toStartOfInterval.cpp @@ -23,11 +23,11 @@ namespace { static constexpr auto function_name = "toStartOfInterval"; - template + template struct Transform; template <> - struct Transform + struct Transform { static UInt16 execute(UInt16 d, UInt64 years, const DateLUTImpl & time_zone) { @@ -41,7 +41,7 @@ namespace }; template <> - struct Transform + struct Transform { static UInt16 execute(UInt16 d, UInt64 quarters, const DateLUTImpl & time_zone) { @@ -55,7 +55,7 @@ namespace }; template <> - struct Transform + struct Transform { static UInt16 execute(UInt16 d, UInt64 months, const DateLUTImpl & time_zone) { @@ -69,7 +69,7 @@ namespace }; template <> - struct Transform + struct Transform { static UInt16 execute(UInt16 d, UInt64 weeks, const DateLUTImpl & time_zone) { @@ -83,7 +83,7 @@ namespace }; template <> - struct Transform + struct Transform { static UInt32 execute(UInt16 d, UInt64 days, const DateLUTImpl & time_zone) { @@ -97,7 +97,7 @@ namespace }; template <> - struct Transform + struct Transform { static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } @@ -105,7 +105,7 @@ namespace }; template <> - struct Transform + struct Transform { static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } @@ -116,7 +116,7 @@ namespace }; template <> - struct Transform + struct Transform { static UInt32 execute(UInt16, UInt64, const DateLUTImpl &) { return dateIsNotSupported(function_name); } @@ -163,9 +163,9 @@ public: "Illegal type " + arguments[1].type->getName() + " of argument of function " + getName() + ". Should be an interval of time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - result_type_is_date = (interval_type->getKind() == DataTypeInterval::Year) - || (interval_type->getKind() == DataTypeInterval::Quarter) || (interval_type->getKind() == DataTypeInterval::Month) - || (interval_type->getKind() == DataTypeInterval::Week); + result_type_is_date = (interval_type->getKind() == IntervalKind::Year) + || (interval_type->getKind() == IntervalKind::Quarter) || (interval_type->getKind() == IntervalKind::Month) + || (interval_type->getKind() == IntervalKind::Week); }; auto check_timezone_argument = [&] @@ -177,7 +177,7 @@ public: ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); if (first_argument_is_date && result_type_is_date) throw Exception( - "The timezone argument of function " + getName() + " with interval type " + interval_type->kindToString() + "The timezone argument of function " + getName() + " with interval type " + interval_type->getKind().toString() + " is allowed only when the 1st argument has the type DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); }; @@ -269,28 +269,28 @@ private: switch (interval_type->getKind()) { - case DataTypeInterval::Second: - return execute(time_column, num_units, time_zone); - case DataTypeInterval::Minute: - return execute(time_column, num_units, time_zone); - case DataTypeInterval::Hour: - return execute(time_column, num_units, time_zone); - case DataTypeInterval::Day: - return execute(time_column, num_units, time_zone); - case DataTypeInterval::Week: - return execute(time_column, num_units, time_zone); - case DataTypeInterval::Month: - return execute(time_column, num_units, time_zone); - case DataTypeInterval::Quarter: - return execute(time_column, num_units, time_zone); - case DataTypeInterval::Year: - return execute(time_column, num_units, time_zone); + case IntervalKind::Second: + return execute(time_column, num_units, time_zone); + case IntervalKind::Minute: + return execute(time_column, num_units, time_zone); + case IntervalKind::Hour: + return execute(time_column, num_units, time_zone); + case IntervalKind::Day: + return execute(time_column, num_units, time_zone); + case IntervalKind::Week: + return execute(time_column, num_units, time_zone); + case IntervalKind::Month: + return execute(time_column, num_units, time_zone); + case IntervalKind::Quarter: + return execute(time_column, num_units, time_zone); + case IntervalKind::Year: + return execute(time_column, num_units, time_zone); } __builtin_unreachable(); } - template + template ColumnPtr execute(const ColumnVector & time_column, UInt64 num_units, const DateLUTImpl & time_zone) { const auto & time_data = time_column.getData(); diff --git a/dbms/src/Parsers/CommonParsers.h b/dbms/src/Parsers/CommonParsers.h index 2eafc1c1853..85b5217b617 100644 --- a/dbms/src/Parsers/CommonParsers.h +++ b/dbms/src/Parsers/CommonParsers.h @@ -46,99 +46,6 @@ protected: } }; -class ParserInterval: public IParserBase -{ -public: - enum class IntervalKind - { - Incorrect, - Second, - Minute, - Hour, - Day, - Week, - Month, - Quarter, - Year - }; - - IntervalKind interval_kind; - - ParserInterval() : interval_kind(IntervalKind::Incorrect) {} - - const char * getToIntervalKindFunctionName() - { - switch (interval_kind) - { - case ParserInterval::IntervalKind::Second: - return "toIntervalSecond"; - case ParserInterval::IntervalKind::Minute: - return "toIntervalMinute"; - case ParserInterval::IntervalKind::Hour: - return "toIntervalHour"; - case ParserInterval::IntervalKind::Day: - return "toIntervalDay"; - case ParserInterval::IntervalKind::Week: - return "toIntervalWeek"; - case ParserInterval::IntervalKind::Month: - return "toIntervalMonth"; - case ParserInterval::IntervalKind::Quarter: - return "toIntervalQuarter"; - case ParserInterval::IntervalKind::Year: - return "toIntervalYear"; - default: - return nullptr; - } - } - -protected: - const char * getName() const override { return "interval"; } - - bool parseImpl(Pos & pos, ASTPtr & /*node*/, Expected & expected) override - { - if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected) - || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) - interval_kind = IntervalKind::Second; - else if ( - ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected) - || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) - interval_kind = IntervalKind::Minute; - else if ( - ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected) - || ParserKeyword("HH").ignore(pos, expected)) - interval_kind = IntervalKind::Hour; - else if ( - ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected) - || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) - interval_kind = IntervalKind::Day; - else if ( - ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected) - || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) - interval_kind = IntervalKind::Week; - else if ( - ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected) - || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) - interval_kind = IntervalKind::Month; - else if ( - ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected) - || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) - interval_kind = IntervalKind::Quarter; - else if ( - ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected) - || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) - interval_kind = IntervalKind::Year; - else - interval_kind = IntervalKind::Incorrect; - - if (interval_kind == IntervalKind::Incorrect) - { - expected.add(pos, "YEAR, QUARTER, MONTH, WEEK, DAY, HOUR, MINUTE or SECOND"); - return false; - } - /// one of ParserKeyword already made ++pos - return true; - } -}; // Parser always returns true and do nothing. class ParserNothing : public IParserBase diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 70be49cdd04..1f1ba4edee7 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -20,7 +20,7 @@ #include #include -#include +#include #include #include #include @@ -690,44 +690,11 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp ++pos; ASTPtr expr; - const char * function_name = nullptr; - ParserInterval interval_parser; - if (!interval_parser.ignore(pos, expected)) + IntervalKind interval_kind; + if (!parseIntervalKind(pos, expected, interval_kind)) return false; - switch (interval_parser.interval_kind) - { - case ParserInterval::IntervalKind::Second: - function_name = "toSecond"; - break; - case ParserInterval::IntervalKind::Minute: - function_name = "toMinute"; - break; - case ParserInterval::IntervalKind::Hour: - function_name = "toHour"; - break; - case ParserInterval::IntervalKind::Day: - function_name = "toDayOfMonth"; - break; - case ParserInterval::IntervalKind::Week: - // TODO: SELECT toRelativeWeekNum(toDate('2017-06-15')) - toRelativeWeekNum(toStartOfYear(toDate('2017-06-15'))) - // else if (ParserKeyword("WEEK").ignore(pos, expected)) - // function_name = "toRelativeWeekNum"; - return false; - case ParserInterval::IntervalKind::Month: - function_name = "toMonth"; - break; - case ParserInterval::IntervalKind::Quarter: - function_name = "toQuarter"; - break; - case ParserInterval::IntervalKind::Year: - function_name = "toYear"; - break; - default: - return false; - } - ParserKeyword s_from("FROM"); if (!s_from.ignore(pos, expected)) return false; @@ -742,7 +709,7 @@ bool ParserExtractExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp auto function = std::make_shared(); auto exp_list = std::make_shared(); - function->name = function_name; //"toYear"; + function->name = interval_kind.toNameOfFunctionExtractTimePart(); function->arguments = exp_list; function->children.push_back(exp_list); exp_list->children.push_back(expr); @@ -770,8 +737,8 @@ bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp return false; ++pos; - ParserInterval interval_parser; - if (interval_parser.ignore(pos, expected)) + IntervalKind interval_kind; + if (parseIntervalKind(pos, expected, interval_kind)) { /// function(unit, offset, timestamp) if (pos->type != TokenType::Comma) @@ -804,20 +771,18 @@ bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp if (!ParserExpression().parse(pos, offset_node, expected)) return false; - interval_parser.ignore(pos, expected); - + if (!parseIntervalKind(pos, expected, interval_kind)) + return false; } if (pos->type != TokenType::ClosingRoundBracket) return false; ++pos; - const char * interval_function_name = interval_parser.getToIntervalKindFunctionName(); - auto interval_expr_list_args = std::make_shared(); interval_expr_list_args->children = {offset_node}; auto interval_func_node = std::make_shared(); - interval_func_node->name = interval_function_name; + interval_func_node->name = interval_kind.toNameOfFunctionToIntervalDataType(); interval_func_node->arguments = std::move(interval_expr_list_args); interval_func_node->children.push_back(interval_func_node->arguments); @@ -836,7 +801,6 @@ bool ParserDateAddExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & exp bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - const char * interval_name = nullptr; ASTPtr left_node; ASTPtr right_node; @@ -848,40 +812,10 @@ bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & ex return false; ++pos; - ParserInterval interval_parser; - if (!interval_parser.ignore(pos, expected)) + IntervalKind interval_kind; + if (!parseIntervalKind(pos, expected, interval_kind)) return false; - switch (interval_parser.interval_kind) - { - case ParserInterval::IntervalKind::Second: - interval_name = "second"; - break; - case ParserInterval::IntervalKind::Minute: - interval_name = "minute"; - break; - case ParserInterval::IntervalKind::Hour: - interval_name = "hour"; - break; - case ParserInterval::IntervalKind::Day: - interval_name = "day"; - break; - case ParserInterval::IntervalKind::Week: - interval_name = "week"; - break; - case ParserInterval::IntervalKind::Month: - interval_name = "month"; - break; - case ParserInterval::IntervalKind::Quarter: - interval_name = "quarter"; - break; - case ParserInterval::IntervalKind::Year: - interval_name = "year"; - break; - default: - return false; - } - if (pos->type != TokenType::Comma) return false; ++pos; @@ -901,7 +835,7 @@ bool ParserDateDiffExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & ex ++pos; auto expr_list_args = std::make_shared(); - expr_list_args->children = {std::make_shared(interval_name), left_node, right_node}; + expr_list_args->children = {std::make_shared(interval_kind.toDateDiffUnit()), left_node, right_node}; auto func_node = std::make_shared(); func_node->name = "dateDiff"; diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index 6d33368d88b..060d1e89f02 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -1,7 +1,7 @@ #include #include #include -#include +#include #include #include #include @@ -604,13 +604,10 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec if (!ParserExpressionWithOptionalAlias(false).parse(pos, expr, expected)) return false; - - ParserInterval interval_parser; - if (!interval_parser.ignore(pos, expected)) + IntervalKind interval_kind; + if (!parseIntervalKind(pos, expected, interval_kind)) return false; - const char * function_name = interval_parser.getToIntervalKindFunctionName(); - /// the function corresponding to the operator auto function = std::make_shared(); @@ -618,7 +615,7 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec auto exp_list = std::make_shared(); /// the first argument of the function is the previous element, the second is the next one - function->name = function_name; + function->name = interval_kind.toNameOfFunctionToIntervalDataType(); function->arguments = exp_list; function->children.push_back(exp_list); diff --git a/dbms/src/Parsers/parseIntervalKind.cpp b/dbms/src/Parsers/parseIntervalKind.cpp new file mode 100644 index 00000000000..7d36133e81c --- /dev/null +++ b/dbms/src/Parsers/parseIntervalKind.cpp @@ -0,0 +1,68 @@ +#include +#include +#include + + +namespace DB +{ +bool parseIntervalKind(IParser::Pos & pos, Expected & expected, IntervalKind & result) +{ + if (ParserKeyword("SECOND").ignore(pos, expected) || ParserKeyword("SQL_TSI_SECOND").ignore(pos, expected) + || ParserKeyword("SS").ignore(pos, expected) || ParserKeyword("S").ignore(pos, expected)) + { + result = IntervalKind::Second; + return true; + } + + if (ParserKeyword("MINUTE").ignore(pos, expected) || ParserKeyword("SQL_TSI_MINUTE").ignore(pos, expected) + || ParserKeyword("MI").ignore(pos, expected) || ParserKeyword("N").ignore(pos, expected)) + { + result = IntervalKind::Minute; + return true; + } + + if (ParserKeyword("HOUR").ignore(pos, expected) || ParserKeyword("SQL_TSI_HOUR").ignore(pos, expected) + || ParserKeyword("HH").ignore(pos, expected)) + { + result = IntervalKind::Hour; + return true; + } + + if (ParserKeyword("DAY").ignore(pos, expected) || ParserKeyword("SQL_TSI_DAY").ignore(pos, expected) + || ParserKeyword("DD").ignore(pos, expected) || ParserKeyword("D").ignore(pos, expected)) + { + result = IntervalKind::Day; + return true; + } + + if (ParserKeyword("WEEK").ignore(pos, expected) || ParserKeyword("SQL_TSI_WEEK").ignore(pos, expected) + || ParserKeyword("WK").ignore(pos, expected) || ParserKeyword("WW").ignore(pos, expected)) + { + result = IntervalKind::Week; + return true; + } + + if (ParserKeyword("MONTH").ignore(pos, expected) || ParserKeyword("SQL_TSI_MONTH").ignore(pos, expected) + || ParserKeyword("MM").ignore(pos, expected) || ParserKeyword("M").ignore(pos, expected)) + { + result = IntervalKind::Month; + return true; + } + + if (ParserKeyword("QUARTER").ignore(pos, expected) || ParserKeyword("SQL_TSI_QUARTER").ignore(pos, expected) + || ParserKeyword("QQ").ignore(pos, expected) || ParserKeyword("Q").ignore(pos, expected)) + { + result = IntervalKind::Quarter; + return true; + } + + if (ParserKeyword("YEAR").ignore(pos, expected) || ParserKeyword("SQL_TSI_YEAR").ignore(pos, expected) + || ParserKeyword("YYYY").ignore(pos, expected) || ParserKeyword("YY").ignore(pos, expected)) + { + result = IntervalKind::Year; + return true; + } + + return false; +} +} diff --git a/dbms/src/Parsers/parseIntervalKind.h b/dbms/src/Parsers/parseIntervalKind.h new file mode 100644 index 00000000000..59f2824dfe2 --- /dev/null +++ b/dbms/src/Parsers/parseIntervalKind.h @@ -0,0 +1,11 @@ +#pragma once + +#include +#include + + +namespace DB +{ +/// Parses an interval kind. +bool parseIntervalKind(IParser::Pos & pos, Expected & expected, IntervalKind & result); +} From 8035af3a792be0371bc4885d55b75306f2ea5ca7 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 16 Nov 2019 16:54:56 +0300 Subject: [PATCH 73/77] Make Field::getScaleMultiplier() inline to allow using FieldVisitorConvertToNumber in the "clickhouse_parsers" library. --- dbms/src/Common/intExp.h | 11 +++++++++-- dbms/src/Core/DecimalComparison.h | 4 ++-- dbms/src/Core/Field.cpp | 15 --------------- dbms/src/Core/Field.h | 2 +- dbms/src/Core/Types.h | 7 +++++++ dbms/src/DataTypes/DataTypesDecimal.cpp | 25 +++---------------------- dbms/src/DataTypes/DataTypesDecimal.h | 18 +++++++++--------- dbms/src/IO/WriteHelpers.h | 9 +-------- 8 files changed, 32 insertions(+), 59 deletions(-) diff --git a/dbms/src/Common/intExp.h b/dbms/src/Common/intExp.h index 163d835819f..0212eb4c084 100644 --- a/dbms/src/Common/intExp.h +++ b/dbms/src/Common/intExp.h @@ -3,11 +3,18 @@ #include #include -#include + +// Also defined in Core/Defines.h +#if !defined(NO_SANITIZE_UNDEFINED) +#if defined(__clang__) + #define NO_SANITIZE_UNDEFINED __attribute__((__no_sanitize__("undefined"))) +#else + #define NO_SANITIZE_UNDEFINED +#endif +#endif /// On overlow, the function returns unspecified value. - inline NO_SANITIZE_UNDEFINED uint64_t intExp2(int x) { return 1ULL << x; diff --git a/dbms/src/Core/DecimalComparison.h b/dbms/src/Core/DecimalComparison.h index bc676ae86c8..cb332ad3779 100644 --- a/dbms/src/Core/DecimalComparison.h +++ b/dbms/src/Core/DecimalComparison.h @@ -88,9 +88,9 @@ public: Shift shift; if (scale_a < scale_b) - shift.a = DataTypeDecimal(maxDecimalPrecision(), scale_b).getScaleMultiplier(scale_b - scale_a); + shift.a = B::getScaleMultiplier(scale_b - scale_a); if (scale_a > scale_b) - shift.b = DataTypeDecimal(maxDecimalPrecision(), scale_a).getScaleMultiplier(scale_a - scale_b); + shift.b = A::getScaleMultiplier(scale_a - scale_b); return applyWithScale(a, b, shift); } diff --git a/dbms/src/Core/Field.cpp b/dbms/src/Core/Field.cpp index 9d27e33c414..505627aaedb 100644 --- a/dbms/src/Core/Field.cpp +++ b/dbms/src/Core/Field.cpp @@ -300,21 +300,6 @@ namespace DB } - template <> Decimal32 DecimalField::getScaleMultiplier() const - { - return DataTypeDecimal::getScaleMultiplier(scale); - } - - template <> Decimal64 DecimalField::getScaleMultiplier() const - { - return DataTypeDecimal::getScaleMultiplier(scale); - } - - template <> Decimal128 DecimalField::getScaleMultiplier() const - { - return DataTypeDecimal::getScaleMultiplier(scale); - } - template static bool decEqual(T x, T y, UInt32 x_scale, UInt32 y_scale) { diff --git a/dbms/src/Core/Field.h b/dbms/src/Core/Field.h index 3d34502c339..885545844f4 100644 --- a/dbms/src/Core/Field.h +++ b/dbms/src/Core/Field.h @@ -102,7 +102,7 @@ public: operator T() const { return dec; } T getValue() const { return dec; } - T getScaleMultiplier() const; + T getScaleMultiplier() const { return T::getScaleMultiplier(scale); } UInt32 getScale() const { return scale; } template diff --git a/dbms/src/Core/Types.h b/dbms/src/Core/Types.h index bbc309aff94..511446b442f 100644 --- a/dbms/src/Core/Types.h +++ b/dbms/src/Core/Types.h @@ -4,6 +4,7 @@ #include #include #include +#include namespace DB @@ -145,6 +146,8 @@ struct Decimal const Decimal & operator /= (const T & x) { value /= x; return *this; } const Decimal & operator %= (const T & x) { value %= x; return *this; } + static T getScaleMultiplier(UInt32 scale); + T value; }; @@ -170,6 +173,10 @@ template <> struct NativeType { using Type = Int32; }; template <> struct NativeType { using Type = Int64; }; template <> struct NativeType { using Type = Int128; }; +template <> inline Int32 Decimal32::getScaleMultiplier(UInt32 scale) { return common::exp10_i32(scale); } +template <> inline Int64 Decimal64::getScaleMultiplier(UInt32 scale) { return common::exp10_i64(scale); } +template <> inline Int128 Decimal128::getScaleMultiplier(UInt32 scale) { return common::exp10_i128(scale); } + inline const char * getTypeName(TypeIndex idx) { switch (idx) diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp index e8caae63a09..84fc31a5ed7 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ b/dbms/src/DataTypes/DataTypesDecimal.cpp @@ -58,7 +58,7 @@ bool DataTypeDecimal::tryReadText(T & x, ReadBuffer & istr, UInt32 precision, { UInt32 unread_scale = scale; bool done = tryReadDecimalText(istr, x, precision, unread_scale); - x *= getScaleMultiplier(unread_scale); + x *= T::getScaleMultiplier(unread_scale); return done; } @@ -70,7 +70,7 @@ void DataTypeDecimal::readText(T & x, ReadBuffer & istr, UInt32 precision, UI readCSVDecimalText(istr, x, precision, unread_scale); else readDecimalText(istr, x, precision, unread_scale); - x *= getScaleMultiplier(unread_scale); + x *= T::getScaleMultiplier(unread_scale); } template @@ -96,7 +96,7 @@ T DataTypeDecimal::parseFromString(const String & str) const T x; UInt32 unread_scale = scale; readDecimalText(buf, x, precision, unread_scale, true); - x *= getScaleMultiplier(unread_scale); + x *= T::getScaleMultiplier(unread_scale); return x; } @@ -271,25 +271,6 @@ void registerDataTypeDecimal(DataTypeFactory & factory) } -template <> -Decimal32 DataTypeDecimal::getScaleMultiplier(UInt32 scale_) -{ - return decimalScaleMultiplier(scale_); -} - -template <> -Decimal64 DataTypeDecimal::getScaleMultiplier(UInt32 scale_) -{ - return decimalScaleMultiplier(scale_); -} - -template <> -Decimal128 DataTypeDecimal::getScaleMultiplier(UInt32 scale_) -{ - return decimalScaleMultiplier(scale_); -} - - /// Explicit template instantiations. template class DataTypeDecimal; template class DataTypeDecimal; diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h index e59a2b6e3fd..8de80050bae 100644 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ b/dbms/src/DataTypes/DataTypesDecimal.h @@ -130,7 +130,7 @@ public: UInt32 getPrecision() const { return precision; } UInt32 getScale() const { return scale; } - T getScaleMultiplier() const { return getScaleMultiplier(scale); } + T getScaleMultiplier() const { return T::getScaleMultiplier(scale); } T wholePart(T x) const { @@ -148,7 +148,7 @@ public: return x % getScaleMultiplier(); } - T maxWholeValue() const { return getScaleMultiplier(maxPrecision() - scale) - T(1); } + T maxWholeValue() const { return T::getScaleMultiplier(maxPrecision() - scale) - T(1); } bool canStoreWhole(T x) const { @@ -165,7 +165,7 @@ public: if (getScale() < x.getScale()) throw Exception("Decimal result's scale is less then argiment's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND); UInt32 scale_delta = getScale() - x.getScale(); /// scale_delta >= 0 - return getScaleMultiplier(scale_delta); + return T::getScaleMultiplier(scale_delta); } template @@ -181,7 +181,6 @@ public: void readText(T & x, ReadBuffer & istr, bool csv = false) const { readText(x, istr, precision, scale, csv); } static void readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale, bool csv = false); static bool tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale); - static T getScaleMultiplier(UInt32 scale); private: const UInt32 precision; @@ -264,12 +263,12 @@ convertDecimals(const typename FromDataType::FieldType & value, UInt32 scale_fro MaxNativeType converted_value; if (scale_to > scale_from) { - converted_value = DataTypeDecimal::getScaleMultiplier(scale_to - scale_from); + converted_value = MaxFieldType::getScaleMultiplier(scale_to - scale_from); if (common::mulOverflow(static_cast(value), converted_value, converted_value)) throw Exception("Decimal convert overflow", ErrorCodes::DECIMAL_OVERFLOW); } else - converted_value = value / DataTypeDecimal::getScaleMultiplier(scale_from - scale_to); + converted_value = value / MaxFieldType::getScaleMultiplier(scale_from - scale_to); if constexpr (sizeof(FromFieldType) > sizeof(ToFieldType)) { @@ -289,7 +288,7 @@ convertFromDecimal(const typename FromDataType::FieldType & value, UInt32 scale) using ToFieldType = typename ToDataType::FieldType; if constexpr (std::is_floating_point_v) - return static_cast(value) / FromDataType::getScaleMultiplier(scale); + return static_cast(value) / FromFieldType::getScaleMultiplier(scale); else { FromFieldType converted_value = convertDecimals(value, scale, 0); @@ -320,14 +319,15 @@ inline std::enable_if_t && IsDataTypeDecimal) { if (!std::isfinite(value)) throw Exception("Decimal convert overflow. Cannot convert infinity or NaN to decimal", ErrorCodes::DECIMAL_OVERFLOW); - auto out = value * ToDataType::getScaleMultiplier(scale); + auto out = value * ToFieldType::getScaleMultiplier(scale); if constexpr (std::is_same_v) { static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64; diff --git a/dbms/src/IO/WriteHelpers.h b/dbms/src/IO/WriteHelpers.h index 509c37257ad..1f5b6ca7f42 100644 --- a/dbms/src/IO/WriteHelpers.h +++ b/dbms/src/IO/WriteHelpers.h @@ -18,7 +18,6 @@ #include #include #include -#include #include #include @@ -764,12 +763,6 @@ inline void writeText(const LocalDateTime & x, WriteBuffer & buf) { writeDateTim inline void writeText(const UUID & x, WriteBuffer & buf) { writeUUIDText(x, buf); } inline void writeText(const UInt128 & x, WriteBuffer & buf) { writeText(UUID(x), buf); } -template inline T decimalScaleMultiplier(UInt32 scale); -template <> inline Int32 decimalScaleMultiplier(UInt32 scale) { return common::exp10_i32(scale); } -template <> inline Int64 decimalScaleMultiplier(UInt32 scale) { return common::exp10_i64(scale); } -template <> inline Int128 decimalScaleMultiplier(UInt32 scale) { return common::exp10_i128(scale); } - - template void writeText(Decimal value, UInt32 scale, WriteBuffer & ostr) { @@ -781,7 +774,7 @@ void writeText(Decimal value, UInt32 scale, WriteBuffer & ostr) T whole_part = value; if (scale) - whole_part = value / decimalScaleMultiplier(scale); + whole_part = value / Decimal::getScaleMultiplier(scale); writeIntText(whole_part, ostr); if (scale) From 6306e2ab918aa424e2a60ca84b9f56f745b3e738 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Mon, 2 Dec 2019 01:01:05 +0300 Subject: [PATCH 74/77] Add DCL to manage quotas. --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Interpreters/Context.cpp | 9 + dbms/src/Interpreters/Context.h | 2 + .../InterpreterCreateQuotaQuery.cpp | 121 ++++++++ .../InterpreterCreateQuotaQuery.h | 29 ++ .../InterpreterDropAccessEntityQuery.cpp | 31 +++ .../InterpreterDropAccessEntityQuery.h | 20 ++ dbms/src/Interpreters/InterpreterFactory.cpp | 24 ++ ...InterpreterShowCreateAccessEntityQuery.cpp | 89 ++++++ .../InterpreterShowCreateAccessEntityQuery.h | 35 +++ .../InterpreterShowQuotasQuery.cpp | 73 +++++ .../Interpreters/InterpreterShowQuotasQuery.h | 28 ++ dbms/src/Interpreters/Users.cpp | 4 +- dbms/src/Interpreters/Users.h | 2 + dbms/src/Parsers/ASTCreateQuotaQuery.cpp | 142 ++++++++++ dbms/src/Parsers/ASTCreateQuotaQuery.h | 62 +++++ dbms/src/Parsers/ASTDropAccessEntityQuery.cpp | 56 ++++ dbms/src/Parsers/ASTDropAccessEntityQuery.h | 28 ++ dbms/src/Parsers/ASTRoleList.cpp | 56 ++++ dbms/src/Parsers/ASTRoleList.h | 25 ++ .../ASTShowCreateAccessEntityQuery.cpp | 51 ++++ .../Parsers/ASTShowCreateAccessEntityQuery.h | 30 ++ dbms/src/Parsers/ASTShowQuotasQuery.cpp | 35 +++ dbms/src/Parsers/ASTShowQuotasQuery.h | 24 ++ dbms/src/Parsers/IParserBase.cpp | 17 +- dbms/src/Parsers/IParserBase.h | 24 ++ dbms/src/Parsers/ParserCreateQuotaQuery.cpp | 261 ++++++++++++++++++ dbms/src/Parsers/ParserCreateQuotaQuery.h | 31 +++ .../Parsers/ParserDropAccessEntityQuery.cpp | 45 +++ .../src/Parsers/ParserDropAccessEntityQuery.h | 17 ++ dbms/src/Parsers/ParserQuery.cpp | 8 +- dbms/src/Parsers/ParserQueryWithOutput.cpp | 8 +- dbms/src/Parsers/ParserRoleList.cpp | 78 ++++++ dbms/src/Parsers/ParserRoleList.h | 18 ++ .../ParserShowCreateAccessEntityQuery.cpp | 47 ++++ .../ParserShowCreateAccessEntityQuery.h | 17 ++ dbms/src/Parsers/ParserShowQuotasQuery.cpp | 42 +++ dbms/src/Parsers/ParserShowQuotasQuery.h | 18 ++ .../integration/test_quota/configs/users.xml | 1 + dbms/tests/integration/test_quota/test.py | 74 +++++ .../0_stateless/01033_quota_dcl.reference | 2 + .../queries/0_stateless/01033_quota_dcl.sql | 3 + 42 files changed, 1674 insertions(+), 14 deletions(-) create mode 100644 dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterCreateQuotaQuery.h create mode 100644 dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterDropAccessEntityQuery.h create mode 100644 dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h create mode 100644 dbms/src/Interpreters/InterpreterShowQuotasQuery.cpp create mode 100644 dbms/src/Interpreters/InterpreterShowQuotasQuery.h create mode 100644 dbms/src/Parsers/ASTCreateQuotaQuery.cpp create mode 100644 dbms/src/Parsers/ASTCreateQuotaQuery.h create mode 100644 dbms/src/Parsers/ASTDropAccessEntityQuery.cpp create mode 100644 dbms/src/Parsers/ASTDropAccessEntityQuery.h create mode 100644 dbms/src/Parsers/ASTRoleList.cpp create mode 100644 dbms/src/Parsers/ASTRoleList.h create mode 100644 dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp create mode 100644 dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h create mode 100644 dbms/src/Parsers/ASTShowQuotasQuery.cpp create mode 100644 dbms/src/Parsers/ASTShowQuotasQuery.h create mode 100644 dbms/src/Parsers/ParserCreateQuotaQuery.cpp create mode 100644 dbms/src/Parsers/ParserCreateQuotaQuery.h create mode 100644 dbms/src/Parsers/ParserDropAccessEntityQuery.cpp create mode 100644 dbms/src/Parsers/ParserDropAccessEntityQuery.h create mode 100644 dbms/src/Parsers/ParserRoleList.cpp create mode 100644 dbms/src/Parsers/ParserRoleList.h create mode 100644 dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp create mode 100644 dbms/src/Parsers/ParserShowCreateAccessEntityQuery.h create mode 100644 dbms/src/Parsers/ParserShowQuotasQuery.cpp create mode 100644 dbms/src/Parsers/ParserShowQuotasQuery.h create mode 100644 dbms/tests/queries/0_stateless/01033_quota_dcl.reference create mode 100644 dbms/tests/queries/0_stateless/01033_quota_dcl.sql diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 5fba98b7a69..d780041f5e2 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -471,6 +471,7 @@ namespace ErrorCodes extern const int ACCESS_ENTITY_FOUND_DUPLICATES = 494; extern const int ACCESS_ENTITY_STORAGE_READONLY = 495; extern const int QUOTA_REQUIRES_CLIENT_KEY = 496; + extern const int NOT_ENOUGH_PRIVILEGES = 497; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 1929ba29cce..f303356be34 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -93,6 +93,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; extern const int SCALAR_ALREADY_EXISTS; extern const int UNKNOWN_SCALAR; + extern const int NOT_ENOUGH_PRIVILEGES; } @@ -599,6 +600,13 @@ const AccessControlManager & Context::getAccessControlManager() const return shared->access_control_manager; } +void Context::checkQuotaManagementIsAllowed() +{ + if (!is_quota_management_allowed) + throw Exception( + "User " + client_info.current_user + " doesn't have enough privileges to manage quotas", ErrorCodes::NOT_ENOUGH_PRIVILEGES); +} + void Context::setUsersConfig(const ConfigurationPtr & config) { auto lock = getLock(); @@ -664,6 +672,7 @@ void Context::calculateUserSettings() quota = getAccessControlManager().createQuotaContext( client_info.current_user, client_info.current_address.host(), client_info.quota_key); + is_quota_management_allowed = user->is_quota_management_allowed; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index a204aab5bc5..c3671990dc3 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -139,6 +139,7 @@ private: InputBlocksReader input_blocks_reader; std::shared_ptr quota; /// Current quota. By default - empty quota, that have no limits. + bool is_quota_management_allowed = false; /// Whether the current user is allowed to manage quotas via SQL commands. String current_database; Settings settings; /// Setting for query execution. std::shared_ptr settings_constraints; @@ -205,6 +206,7 @@ public: AccessControlManager & getAccessControlManager(); const AccessControlManager & getAccessControlManager() const; std::shared_ptr getQuota() const { return quota; } + void checkQuotaManagementIsAllowed(); /** Take the list of users, quotas and configuration profiles from this config. * The list of users is completely replaced. diff --git a/dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp new file mode 100644 index 00000000000..0dd81f5cb27 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateQuotaQuery.cpp @@ -0,0 +1,121 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +BlockIO InterpreterCreateQuotaQuery::execute() +{ + context.checkQuotaManagementIsAllowed(); + const auto & query = query_ptr->as(); + auto & access_control = context.getAccessControlManager(); + + if (query.alter) + { + auto update_func = [&](const AccessEntityPtr & entity) -> AccessEntityPtr + { + auto updated_quota = typeid_cast>(entity->clone()); + updateQuotaFromQuery(*updated_quota, query); + return updated_quota; + }; + if (query.if_exists) + { + if (auto id = access_control.find(query.name)) + access_control.tryUpdate(*id, update_func); + } + else + access_control.update(access_control.getID(query.name), update_func); + } + else + { + auto new_quota = std::make_shared(); + updateQuotaFromQuery(*new_quota, query); + + if (query.if_not_exists) + access_control.tryInsert(new_quota); + else if (query.or_replace) + access_control.insertOrReplace(new_quota); + else + access_control.insert(new_quota); + } + + return {}; +} + + +void InterpreterCreateQuotaQuery::updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query) +{ + if (query.alter) + { + if (!query.new_name.empty()) + quota.setName(query.new_name); + } + else + quota.setName(query.name); + + if (query.key_type) + quota.key_type = *query.key_type; + + auto & quota_all_limits = quota.all_limits; + for (const auto & query_limits : query.all_limits) + { + auto duration = query_limits.duration; + + auto it = boost::range::find_if(quota_all_limits, [&](const Quota::Limits & x) { return x.duration == duration; }); + if (query_limits.unset_tracking) + { + if (it != quota_all_limits.end()) + quota_all_limits.erase(it); + continue; + } + + if (it == quota_all_limits.end()) + { + /// We keep `all_limits` sorted by duration. + it = quota_all_limits.insert( + boost::range::upper_bound( + quota_all_limits, + duration, + [](const std::chrono::seconds & lhs, const Quota::Limits & rhs) { return lhs < rhs.duration; }), + Quota::Limits{}); + it->duration = duration; + } + + auto & quota_limits = *it; + quota_limits.randomize_interval = query_limits.randomize_interval; + for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE)) + { + if (query_limits.max[resource_type]) + quota_limits.max[resource_type] = *query_limits.max[resource_type]; + } + } + + if (query.roles) + { + const auto & query_roles = *query.roles; + + /// We keep `roles` sorted. + quota.roles = query_roles.roles; + if (query_roles.current_user) + quota.roles.push_back(context.getClientInfo().current_user); + boost::range::sort(quota.roles); + quota.roles.erase(std::unique(quota.roles.begin(), quota.roles.end()), quota.roles.end()); + + quota.all_roles = query_roles.all_roles; + + /// We keep `except_roles` sorted. + quota.except_roles = query_roles.except_roles; + if (query_roles.except_current_user) + quota.except_roles.push_back(context.getClientInfo().current_user); + boost::range::sort(quota.except_roles); + quota.except_roles.erase(std::unique(quota.except_roles.begin(), quota.except_roles.end()), quota.except_roles.end()); + } +} +} diff --git a/dbms/src/Interpreters/InterpreterCreateQuotaQuery.h b/dbms/src/Interpreters/InterpreterCreateQuotaQuery.h new file mode 100644 index 00000000000..bbf91bbe1d3 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterCreateQuotaQuery.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTCreateQuotaQuery; +struct Quota; + + +class InterpreterCreateQuotaQuery : public IInterpreter +{ +public: + InterpreterCreateQuotaQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override; + + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + +private: + void updateQuotaFromQuery(Quota & quota, const ASTCreateQuotaQuery & query); + + ASTPtr query_ptr; + Context & context; +}; +} diff --git a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp new file mode 100644 index 00000000000..7f18084038c --- /dev/null +++ b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.cpp @@ -0,0 +1,31 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +BlockIO InterpreterDropAccessEntityQuery::execute() +{ + const auto & query = query_ptr->as(); + auto & access_control = context.getAccessControlManager(); + using Kind = ASTDropAccessEntityQuery::Kind; + + switch (query.kind) + { + case Kind::QUOTA: + { + context.checkQuotaManagementIsAllowed(); + if (query.if_exists) + access_control.tryRemove(access_control.find(query.names)); + else + access_control.remove(access_control.getIDs(query.names)); + return {}; + } + } + + __builtin_unreachable(); +} +} diff --git a/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.h b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.h new file mode 100644 index 00000000000..2a0e749b265 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterDropAccessEntityQuery.h @@ -0,0 +1,20 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class InterpreterDropAccessEntityQuery : public IInterpreter +{ +public: + InterpreterDropAccessEntityQuery(const ASTPtr & query_ptr_, Context & context_) : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override; + +private: + ASTPtr query_ptr; + Context & context; +}; +} diff --git a/dbms/src/Interpreters/InterpreterFactory.cpp b/dbms/src/Interpreters/InterpreterFactory.cpp index d27c9c8baeb..33e9da95dfc 100644 --- a/dbms/src/Interpreters/InterpreterFactory.cpp +++ b/dbms/src/Interpreters/InterpreterFactory.cpp @@ -1,6 +1,8 @@ #include #include #include +#include +#include #include #include #include @@ -9,7 +11,9 @@ #include #include #include +#include #include +#include #include #include #include @@ -19,8 +23,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -31,8 +37,10 @@ #include #include #include +#include #include #include +#include #include #include #include @@ -187,6 +195,22 @@ std::unique_ptr InterpreterFactory::get(ASTPtr & query, Context & { return std::make_unique(query, context); } + else if (query->as()) + { + return std::make_unique(query, context); + } + else if (query->as()) + { + return std::make_unique(query, context); + } + else if (query->as()) + { + return std::make_unique(query, context); + } + else if (query->as()) + { + return std::make_unique(query, context); + } else throw Exception("Unknown type of query: " + query->getID(), ErrorCodes::UNKNOWN_TYPE_OF_QUERY); } diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp new file mode 100644 index 00000000000..d0ef8992691 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.cpp @@ -0,0 +1,89 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +BlockIO InterpreterShowCreateAccessEntityQuery::execute() +{ + BlockIO res; + res.in = executeImpl(); + return res; +} + + +BlockInputStreamPtr InterpreterShowCreateAccessEntityQuery::executeImpl() +{ + const auto & show_query = query_ptr->as(); + + /// Build a create query. + ASTPtr create_query = getCreateQuotaQuery(show_query); + + /// Build the result column. + std::stringstream create_query_ss; + formatAST(*create_query, create_query_ss, false, true); + String create_query_str = create_query_ss.str(); + MutableColumnPtr column = ColumnString::create(); + column->insert(create_query_str); + + /// Prepare description of the result column. + std::stringstream desc_ss; + formatAST(show_query, desc_ss, false, true); + String desc = desc_ss.str(); + String prefix = "SHOW "; + if (startsWith(desc, prefix)) + desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. + + return std::make_shared(Block{{std::move(column), std::make_shared(), desc}}); +} + + +ASTPtr InterpreterShowCreateAccessEntityQuery::getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const +{ + auto & access_control = context.getAccessControlManager(); + + QuotaPtr quota; + if (show_query.current_quota) + quota = access_control.read(context.getQuota()->getUsageInfo().quota_id); + else + quota = access_control.read(show_query.name); + + auto create_query = std::make_shared(); + create_query->name = quota->getName(); + create_query->key_type = quota->key_type; + create_query->all_limits.reserve(quota->all_limits.size()); + + for (const auto & limits : quota->all_limits) + { + ASTCreateQuotaQuery::Limits create_query_limits; + create_query_limits.duration = limits.duration; + create_query_limits.randomize_interval = limits.randomize_interval; + for (auto resource_type : ext::range(Quota::MAX_RESOURCE_TYPE)) + if (limits.max[resource_type]) + create_query_limits.max[resource_type] = limits.max[resource_type]; + create_query->all_limits.push_back(create_query_limits); + } + + if (!quota->roles.empty() || quota->all_roles) + { + auto create_query_roles = std::make_shared(); + create_query_roles->roles = quota->roles; + create_query_roles->all_roles = quota->all_roles; + create_query_roles->except_roles = quota->except_roles; + create_query->roles = std::move(create_query_roles); + } + + return create_query; +} +} diff --git a/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h new file mode 100644 index 00000000000..94b06dadb19 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowCreateAccessEntityQuery.h @@ -0,0 +1,35 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class Context; +class ASTShowCreateAccessEntityQuery; + + +/** Returns a single item containing a statement which could be used to create a specified role. + */ +class InterpreterShowCreateAccessEntityQuery : public IInterpreter +{ +public: + InterpreterShowCreateAccessEntityQuery(const ASTPtr & query_ptr_, const Context & context_) + : query_ptr(query_ptr_), context(context_) {} + + BlockIO execute() override; + + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + +private: + ASTPtr query_ptr; + const Context & context; + + BlockInputStreamPtr executeImpl(); + ASTPtr getCreateQuotaQuery(const ASTShowCreateAccessEntityQuery & show_query) const; +}; + + +} diff --git a/dbms/src/Interpreters/InterpreterShowQuotasQuery.cpp b/dbms/src/Interpreters/InterpreterShowQuotasQuery.cpp new file mode 100644 index 00000000000..73653e26781 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowQuotasQuery.cpp @@ -0,0 +1,73 @@ +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +InterpreterShowQuotasQuery::InterpreterShowQuotasQuery(const ASTPtr & query_ptr_, Context & context_) + : query_ptr(query_ptr_), context(context_) +{ +} + + +String InterpreterShowQuotasQuery::getRewrittenQuery() +{ + const auto & query = query_ptr->as(); + + /// Transform the query into some kind of "SELECT from system.quotas" query. + String expr; + String filter; + String table_name; + String order_by; + if (query.usage) + { + expr = "name || ' key=\\'' || key || '\\'' || if(isNull(end_of_interval), '', ' interval=[' || " + "toString(end_of_interval - duration) || ' .. ' || " + "toString(end_of_interval) || ']'"; + for (auto resource_type : ext::range_with_static_cast(Quota::MAX_RESOURCE_TYPE)) + { + String column_name = Quota::resourceTypeToColumnName(resource_type); + expr += String{" || ' "} + column_name + "=' || toString(" + column_name + ")"; + expr += String{" || if(max_"} + column_name + "=0, '', '/' || toString(max_" + column_name + "))"; + } + expr += ")"; + + if (query.current) + filter = "(id = currentQuotaID()) AND (key = currentQuotaKey())"; + + table_name = "system.quota_usage"; + order_by = "name, key, duration"; + } + else + { + expr = "name"; + table_name = "system.quotas"; + order_by = "name"; + } + + /// Prepare description of the result column. + std::stringstream ss; + formatAST(query, ss, false, true); + String desc = ss.str(); + String prefix = "SHOW "; + if (startsWith(desc, prefix)) + desc = desc.substr(prefix.length()); /// `desc` always starts with "SHOW ", so we can trim this prefix. + + /// Build a new query. + return "SELECT " + expr + " AS " + backQuote(desc) + " FROM " + table_name + (filter.empty() ? "" : (" WHERE " + filter)) + + (order_by.empty() ? "" : (" ORDER BY " + order_by)); +} + + +BlockIO InterpreterShowQuotasQuery::execute() +{ + return executeQuery(getRewrittenQuery(), context, true); +} + +} diff --git a/dbms/src/Interpreters/InterpreterShowQuotasQuery.h b/dbms/src/Interpreters/InterpreterShowQuotasQuery.h new file mode 100644 index 00000000000..ae608e81ce5 --- /dev/null +++ b/dbms/src/Interpreters/InterpreterShowQuotasQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class Context; + +class InterpreterShowQuotasQuery : public IInterpreter +{ +public: + InterpreterShowQuotasQuery(const ASTPtr & query_ptr_, Context & context_); + + BlockIO execute() override; + + bool ignoreQuota() const override { return true; } + bool ignoreLimits() const override { return true; } + +private: + ASTPtr query_ptr; + Context & context; + + String getRewrittenQuery(); +}; + +} diff --git a/dbms/src/Interpreters/Users.cpp b/dbms/src/Interpreters/Users.cpp index 7ac738031c2..e66b5119f84 100644 --- a/dbms/src/Interpreters/Users.cpp +++ b/dbms/src/Interpreters/Users.cpp @@ -129,7 +129,9 @@ User::User(const String & name_, const String & config_elem, const Poco::Util::A } } } + + if (config.has(config_elem + ".allow_quota_management")) + is_quota_management_allowed = config.getBool(config_elem + ".allow_quota_management"); } - } diff --git a/dbms/src/Interpreters/Users.h b/dbms/src/Interpreters/Users.h index b7340fadcfa..6f9a47c4422 100644 --- a/dbms/src/Interpreters/Users.h +++ b/dbms/src/Interpreters/Users.h @@ -47,6 +47,8 @@ struct User using DatabaseMap = std::unordered_map; DatabaseMap table_props; + bool is_quota_management_allowed = false; + User(const String & name_, const String & config_elem, const Poco::Util::AbstractConfiguration & config); }; diff --git a/dbms/src/Parsers/ASTCreateQuotaQuery.cpp b/dbms/src/Parsers/ASTCreateQuotaQuery.cpp new file mode 100644 index 00000000000..2814515d61f --- /dev/null +++ b/dbms/src/Parsers/ASTCreateQuotaQuery.cpp @@ -0,0 +1,142 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace +{ + using KeyType = Quota::KeyType; + using ResourceType = Quota::ResourceType; + using ResourceAmount = Quota::ResourceAmount; + + + void formatKeyType(const KeyType & key_type, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " KEYED BY " << (settings.hilite ? IAST::hilite_none : "") << "'" + << Quota::getNameOfKeyType(key_type) << "'"; + } + + + void formatRenameTo(const String & new_name, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " RENAME TO " << (settings.hilite ? IAST::hilite_none : "") + << backQuote(new_name); + } + + + void formatLimit(ResourceType resource_type, ResourceAmount max, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " MAX " << Quota::resourceTypeToKeyword(resource_type) + << (settings.hilite ? IAST::hilite_none : ""); + + settings.ostr << (settings.hilite ? IAST::hilite_operator : "") << " = " << (settings.hilite ? IAST::hilite_none : ""); + + if (max == Quota::UNLIMITED) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ANY" << (settings.hilite ? IAST::hilite_none : ""); + else if (resource_type == Quota::EXECUTION_TIME) + settings.ostr << Quota::executionTimeToSeconds(max); + else + settings.ostr << max; + } + + + void formatLimits(const ASTCreateQuotaQuery::Limits & limits, const IAST::FormatSettings & settings) + { + auto interval_kind = IntervalKind::fromAvgSeconds(limits.duration.count()); + Int64 num_intervals = limits.duration.count() / interval_kind.toAvgSeconds(); + + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") + << " FOR" + << (limits.randomize_interval ? " RANDOMIZED" : "") + << " INTERVAL " + << (settings.hilite ? IAST::hilite_none : "") + << num_intervals << " " + << (settings.hilite ? IAST::hilite_keyword : "") + << interval_kind.toKeyword() + << (settings.hilite ? IAST::hilite_none : ""); + + if (limits.unset_tracking) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " UNSET TRACKING" << (settings.hilite ? IAST::hilite_none : ""); + } + else + { + bool limit_found = false; + for (auto resource_type : ext::range_with_static_cast(Quota::MAX_RESOURCE_TYPE)) + { + if (limits.max[resource_type]) + { + if (limit_found) + settings.ostr << ","; + limit_found = true; + formatLimit(resource_type, *limits.max[resource_type], settings); + } + } + if (!limit_found) + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TRACKING" << (settings.hilite ? IAST::hilite_none : ""); + } + } + + void formatAllLimits(const std::vector & all_limits, const IAST::FormatSettings & settings) + { + bool need_comma = false; + for (auto & limits : all_limits) + { + if (need_comma) + settings.ostr << ","; + need_comma = true; + + formatLimits(limits, settings); + } + } + + void formatRoles(const ASTRoleList & roles, const IAST::FormatSettings & settings) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " TO " << (settings.hilite ? IAST::hilite_none : ""); + roles.format(settings); + } +} + + +String ASTCreateQuotaQuery::getID(char) const +{ + return "CreateQuotaQuery"; +} + + +ASTPtr ASTCreateQuotaQuery::clone() const +{ + return std::make_shared(*this); +} + + +void ASTCreateQuotaQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") << (alter ? "ALTER QUOTA" : "CREATE QUOTA") + << (settings.hilite ? hilite_none : ""); + + if (if_exists) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF EXISTS" << (settings.hilite ? hilite_none : ""); + else if (if_not_exists) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " IF NOT EXISTS" << (settings.hilite ? hilite_none : ""); + else if (or_replace) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " OR REPLACE" << (settings.hilite ? hilite_none : ""); + + settings.ostr << " " << backQuoteIfNeed(name); + + if (!new_name.empty()) + formatRenameTo(new_name, settings); + + if (key_type) + formatKeyType(*key_type, settings); + + formatAllLimits(all_limits, settings); + + if (roles) + formatRoles(*roles, settings); +} +} diff --git a/dbms/src/Parsers/ASTCreateQuotaQuery.h b/dbms/src/Parsers/ASTCreateQuotaQuery.h new file mode 100644 index 00000000000..056a445f23b --- /dev/null +++ b/dbms/src/Parsers/ASTCreateQuotaQuery.h @@ -0,0 +1,62 @@ +#pragma once + +#include +#include + + +namespace DB +{ +class ASTRoleList; + + +/** CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name + * [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] + * [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + * {[SET] MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = {number | ANY} } [,...] | + * [SET] TRACKING} [,...]] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + * + * ALTER QUOTA [IF EXISTS] name + * [RENAME TO new_name] + * [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] + * [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + * {[SET] MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = {number | ANY} } [,...] | + * [SET] TRACKING | + * UNSET TRACKING} [,...]] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + */ +class ASTCreateQuotaQuery : public IAST +{ +public: + bool alter = false; + + bool if_exists = false; + bool if_not_exists = false; + bool or_replace = false; + + String name; + String new_name; + + using KeyType = Quota::KeyType; + std::optional key_type; + + using ResourceType = Quota::ResourceType; + using ResourceAmount = Quota::ResourceAmount; + static constexpr size_t MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; + + struct Limits + { + std::optional max[MAX_RESOURCE_TYPE]; + bool unset_tracking = false; + std::chrono::seconds duration = std::chrono::seconds::zero(); + bool randomize_interval = false; + }; + std::vector all_limits; + + std::shared_ptr roles; + + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; +} diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp new file mode 100644 index 00000000000..80d69ed5316 --- /dev/null +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.cpp @@ -0,0 +1,56 @@ +#include +#include + + +namespace DB +{ +namespace +{ + using Kind = ASTDropAccessEntityQuery::Kind; + + const char * kindToKeyword(Kind kind) + { + switch (kind) + { + case Kind::QUOTA: return "QUOTA"; + } + __builtin_unreachable(); + } +} + + +ASTDropAccessEntityQuery::ASTDropAccessEntityQuery(Kind kind_) + : kind(kind_), keyword(kindToKeyword(kind_)) +{ +} + + +String ASTDropAccessEntityQuery::getID(char) const +{ + return String("DROP ") + keyword + " query"; +} + + +ASTPtr ASTDropAccessEntityQuery::clone() const +{ + return std::make_shared(*this); +} + + +void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") + << "DROP " << keyword + << (if_exists ? " IF EXISTS" : "") + << (settings.hilite ? hilite_none : ""); + + bool need_comma = false; + for (const auto & name : names) + { + if (need_comma) + settings.ostr << ','; + need_comma = true; + settings.ostr << ' ' << backQuoteIfNeed(name); + } +} +} diff --git a/dbms/src/Parsers/ASTDropAccessEntityQuery.h b/dbms/src/Parsers/ASTDropAccessEntityQuery.h new file mode 100644 index 00000000000..91b76253db4 --- /dev/null +++ b/dbms/src/Parsers/ASTDropAccessEntityQuery.h @@ -0,0 +1,28 @@ +#pragma once + +#include + + +namespace DB +{ + +/** DROP QUOTA [IF EXISTS] name [,...] + */ +class ASTDropAccessEntityQuery : public IAST +{ +public: + enum class Kind + { + QUOTA, + }; + const Kind kind; + const char * const keyword; + bool if_exists = false; + Strings names; + + ASTDropAccessEntityQuery(Kind kind_); + String getID(char) const override; + ASTPtr clone() const override; + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; +} diff --git a/dbms/src/Parsers/ASTRoleList.cpp b/dbms/src/Parsers/ASTRoleList.cpp new file mode 100644 index 00000000000..9e0a4fffc36 --- /dev/null +++ b/dbms/src/Parsers/ASTRoleList.cpp @@ -0,0 +1,56 @@ +#include +#include + + +namespace DB +{ +void ASTRoleList::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + if (empty()) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "NONE" << (settings.hilite ? IAST::hilite_none : ""); + return; + } + + bool need_comma = false; + if (current_user) + { + if (std::exchange(need_comma, true)) + settings.ostr << ", "; + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "CURRENT_USER" << (settings.hilite ? IAST::hilite_none : ""); + } + + for (auto & role : roles) + { + if (std::exchange(need_comma, true)) + settings.ostr << ", "; + settings.ostr << backQuoteIfNeed(role); + } + + if (all_roles) + { + if (std::exchange(need_comma, true)) + settings.ostr << ", "; + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "ALL" << (settings.hilite ? IAST::hilite_none : ""); + if (except_current_user || !except_roles.empty()) + { + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " EXCEPT " << (settings.hilite ? IAST::hilite_none : ""); + need_comma = false; + + if (except_current_user) + { + if (std::exchange(need_comma, true)) + settings.ostr << ", "; + settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << "CURRENT_USER" << (settings.hilite ? IAST::hilite_none : ""); + } + + for (auto & except_role : except_roles) + { + if (std::exchange(need_comma, true)) + settings.ostr << ", "; + settings.ostr << backQuoteIfNeed(except_role); + } + } + } +} +} diff --git a/dbms/src/Parsers/ASTRoleList.h b/dbms/src/Parsers/ASTRoleList.h new file mode 100644 index 00000000000..5e8859732c2 --- /dev/null +++ b/dbms/src/Parsers/ASTRoleList.h @@ -0,0 +1,25 @@ +#pragma once + +#include +#include + + +namespace DB +{ +/// {role|CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {role|CURRENT_USER} [,...] +class ASTRoleList : public IAST +{ +public: + Strings roles; + bool current_user = false; + bool all_roles = false; + Strings except_roles; + bool except_current_user = false; + + bool empty() const { return roles.empty() && !current_user && !all_roles; } + + String getID(char) const override { return "RoleList"; } + ASTPtr clone() const override { return std::make_shared(*this); } + void formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; +} diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp new file mode 100644 index 00000000000..8509a902014 --- /dev/null +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.cpp @@ -0,0 +1,51 @@ +#include +#include + + +namespace DB +{ +namespace +{ + using Kind = ASTShowCreateAccessEntityQuery::Kind; + + const char * kindToKeyword(Kind kind) + { + switch (kind) + { + case Kind::QUOTA: return "QUOTA"; + } + __builtin_unreachable(); + } +} + + +ASTShowCreateAccessEntityQuery::ASTShowCreateAccessEntityQuery(Kind kind_) + : kind(kind_), keyword(kindToKeyword(kind_)) +{ +} + + +String ASTShowCreateAccessEntityQuery::getID(char) const +{ + return String("SHOW CREATE ") + keyword + " query"; +} + + +ASTPtr ASTShowCreateAccessEntityQuery::clone() const +{ + return std::make_shared(*this); +} + + +void ASTShowCreateAccessEntityQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : "") + << "SHOW CREATE " << keyword + << (settings.hilite ? hilite_none : ""); + + if (current_quota) + settings.ostr << (settings.hilite ? hilite_keyword : "") << " CURRENT" << (settings.hilite ? hilite_none : ""); + else + settings.ostr << " " << backQuoteIfNeed(name); +} +} diff --git a/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h new file mode 100644 index 00000000000..32c0ace101b --- /dev/null +++ b/dbms/src/Parsers/ASTShowCreateAccessEntityQuery.h @@ -0,0 +1,30 @@ +#pragma once + +#include + + +namespace DB +{ +/** SHOW CREATE QUOTA [name | CURRENT] + */ +class ASTShowCreateAccessEntityQuery : public ASTQueryWithOutput +{ +public: + enum class Kind + { + QUOTA, + }; + const Kind kind; + const char * const keyword; + String name; + bool current_quota = false; + + ASTShowCreateAccessEntityQuery(Kind kind_); + String getID(char) const override; + ASTPtr clone() const override; + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +} diff --git a/dbms/src/Parsers/ASTShowQuotasQuery.cpp b/dbms/src/Parsers/ASTShowQuotasQuery.cpp new file mode 100644 index 00000000000..ca7bd5e853f --- /dev/null +++ b/dbms/src/Parsers/ASTShowQuotasQuery.cpp @@ -0,0 +1,35 @@ +#include +#include + + +namespace DB +{ +String ASTShowQuotasQuery::getID(char) const +{ + if (usage) + return "SHOW QUOTA USAGE query"; + else + return "SHOW QUOTAS query"; +} + + +ASTPtr ASTShowQuotasQuery::clone() const +{ + return std::make_shared(*this); +} + + +void ASTShowQuotasQuery::formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const +{ + settings.ostr << (settings.hilite ? hilite_keyword : ""); + + if (usage && current) + settings.ostr << "SHOW QUOTA USAGE"; + else if (usage) + settings.ostr << "SHOW QUOTA USAGE ALL"; + else + settings.ostr << "SHOW QUOTAS"; + + settings.ostr << (settings.hilite ? hilite_none : ""); +} +} diff --git a/dbms/src/Parsers/ASTShowQuotasQuery.h b/dbms/src/Parsers/ASTShowQuotasQuery.h new file mode 100644 index 00000000000..27a08a99a54 --- /dev/null +++ b/dbms/src/Parsers/ASTShowQuotasQuery.h @@ -0,0 +1,24 @@ +#pragma once + +#include + + +namespace DB +{ +/** SHOW QUOTAS + * SHOW QUOTA USAGE [CURRENT | ALL] + */ +class ASTShowQuotasQuery : public ASTQueryWithOutput +{ +public: + bool usage = false; + bool current = false; + + String getID(char) const override; + ASTPtr clone() const override; + +protected: + void formatQueryImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const override; +}; + +} diff --git a/dbms/src/Parsers/IParserBase.cpp b/dbms/src/Parsers/IParserBase.cpp index 64162a595c9..e4caffa992e 100644 --- a/dbms/src/Parsers/IParserBase.cpp +++ b/dbms/src/Parsers/IParserBase.cpp @@ -12,20 +12,15 @@ namespace ErrorCodes bool IParserBase::parse(Pos & pos, ASTPtr & node, Expected & expected) { - Pos begin = pos; expected.add(pos, getName()); - pos.increaseDepth(); - bool res = parseImpl(pos, node, expected); - pos.decreaseDepth(); - - if (!res) + return wrapParseImpl(pos, IncreaseDepthTag{}, [&] { - node = nullptr; - pos = begin; - } - - return res; + bool res = parseImpl(pos, node, expected); + if (!res) + node = nullptr; + return res; + }); } } diff --git a/dbms/src/Parsers/IParserBase.h b/dbms/src/Parsers/IParserBase.h index 67b222b1b71..95951d5acb8 100644 --- a/dbms/src/Parsers/IParserBase.h +++ b/dbms/src/Parsers/IParserBase.h @@ -11,6 +11,30 @@ namespace DB class IParserBase : public IParser { public: + template + static bool wrapParseImpl(Pos & pos, const F & func) + { + Pos begin = pos; + bool res = func(); + if (!res) + pos = begin; + return res; + } + + struct IncreaseDepthTag {}; + + template + static bool wrapParseImpl(Pos & pos, IncreaseDepthTag, const F & func) + { + Pos begin = pos; + pos.increaseDepth(); + bool res = func(); + pos.decreaseDepth(); + if (!res) + pos = begin; + return res; + } + bool parse(Pos & pos, ASTPtr & node, Expected & expected); protected: diff --git a/dbms/src/Parsers/ParserCreateQuotaQuery.cpp b/dbms/src/Parsers/ParserCreateQuotaQuery.cpp new file mode 100644 index 00000000000..cc5fa4bfbcc --- /dev/null +++ b/dbms/src/Parsers/ParserCreateQuotaQuery.cpp @@ -0,0 +1,261 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + + +namespace +{ + using KeyType = Quota::KeyType; + using ResourceType = Quota::ResourceType; + using ResourceAmount = Quota::ResourceAmount; + + bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_name, bool alter) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!new_name.empty() || !alter) + return false; + + if (!ParserKeyword{"RENAME TO"}.ignore(pos, expected)) + return false; + + return parseIdentifierOrStringLiteral(pos, expected, new_name); + }); + } + + bool parseKeyType(IParserBase::Pos & pos, Expected & expected, std::optional & key_type) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (key_type) + return false; + + if (!ParserKeyword{"KEYED BY"}.ignore(pos, expected)) + return false; + + ASTPtr key_type_ast; + if (!ParserStringLiteral().parse(pos, key_type_ast, expected)) + return false; + + const String & key_type_str = key_type_ast->as().value.safeGet(); + for (auto kt : ext::range_with_static_cast(Quota::MAX_KEY_TYPE)) + if (boost::iequals(Quota::getNameOfKeyType(kt), key_type_str)) + { + key_type = kt; + return true; + } + + String all_key_types_str; + for (auto kt : ext::range_with_static_cast(Quota::MAX_KEY_TYPE)) + all_key_types_str += String(all_key_types_str.empty() ? "" : ", ") + "'" + Quota::getNameOfKeyType(kt) + "'"; + String msg = "Quota cannot be keyed by '" + key_type_str + "'. Expected one of these literals: " + all_key_types_str; + throw Exception(msg, ErrorCodes::SYNTAX_ERROR); + }); + } + + bool parseLimit(IParserBase::Pos & pos, Expected & expected, ResourceType & resource_type, ResourceAmount & max) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserKeyword{"MAX"}.ignore(pos, expected)) + return false; + + bool resource_type_set = false; + for (auto rt : ext::range_with_static_cast(Quota::MAX_RESOURCE_TYPE)) + { + if (ParserKeyword{Quota::resourceTypeToKeyword(rt)}.ignore(pos, expected)) + { + resource_type = rt; + resource_type_set = true; + break; + } + } + if (!resource_type_set) + return false; + + if (!ParserToken{TokenType::Equals}.ignore(pos, expected)) + return false; + + ASTPtr max_ast; + if (ParserNumber{}.parse(pos, max_ast, expected)) + { + const Field & max_field = max_ast->as().value; + if (resource_type == Quota::EXECUTION_TIME) + max = Quota::secondsToExecutionTime(applyVisitor(FieldVisitorConvertToNumber(), max_field)); + else + max = applyVisitor(FieldVisitorConvertToNumber(), max_field); + } + else if (ParserKeyword{"ANY"}.ignore(pos, expected)) + { + max = Quota::UNLIMITED; + } + else + return false; + + return true; + }); + } + + bool parseCommaAndLimit(IParserBase::Pos & pos, Expected & expected, ResourceType & resource_type, ResourceAmount & max) + { + return IParserBase::wrapParseImpl(pos, [&] + { + if (!ParserToken{TokenType::Comma}.ignore(pos, expected)) + return false; + + return parseLimit(pos, expected, resource_type, max); + }); + } + + bool parseLimits(IParserBase::Pos & pos, Expected & expected, ASTCreateQuotaQuery::Limits & limits, bool alter) + { + return IParserBase::wrapParseImpl(pos, [&] + { + ASTCreateQuotaQuery::Limits new_limits; + if (!ParserKeyword{"FOR"}.ignore(pos, expected)) + return false; + + new_limits.randomize_interval = ParserKeyword{"RANDOMIZED"}.ignore(pos, expected); + + if (!ParserKeyword{"INTERVAL"}.ignore(pos, expected)) + return false; + + ASTPtr num_intervals_ast; + if (!ParserNumber{}.parse(pos, num_intervals_ast, expected)) + return false; + + double num_intervals = applyVisitor(FieldVisitorConvertToNumber(), num_intervals_ast->as().value); + + IntervalKind interval_kind; + if (!parseIntervalKind(pos, expected, interval_kind)) + return false; + + new_limits.duration = std::chrono::seconds(static_cast(num_intervals * interval_kind.toAvgSeconds())); + + if (alter && ParserKeyword{"UNSET TRACKING"}.ignore(pos, expected)) + { + new_limits.unset_tracking = true; + } + else if (ParserKeyword{"SET TRACKING"}.ignore(pos, expected) || ParserKeyword{"TRACKING"}.ignore(pos, expected)) + { + } + else + { + ParserKeyword{"SET"}.ignore(pos, expected); + ResourceType resource_type; + ResourceAmount max; + if (!parseLimit(pos, expected, resource_type, max)) + return false; + + new_limits.max[resource_type] = max; + while (parseCommaAndLimit(pos, expected, resource_type, max)) + new_limits.max[resource_type] = max; + } + + limits = new_limits; + return true; + }); + } + + bool parseAllLimits(IParserBase::Pos & pos, Expected & expected, std::vector & all_limits, bool alter) + { + return IParserBase::wrapParseImpl(pos, [&] + { + do + { + ASTCreateQuotaQuery::Limits limits; + if (!parseLimits(pos, expected, limits, alter)) + return false; + all_limits.push_back(limits); + } + while (ParserToken{TokenType::Comma}.ignore(pos, expected)); + return true; + }); + } + + bool parseRoles(IParserBase::Pos & pos, Expected & expected, std::shared_ptr & roles) + { + return IParserBase::wrapParseImpl(pos, [&] + { + ASTPtr node; + if (roles || !ParserKeyword{"TO"}.ignore(pos, expected) || !ParserRoleList{}.parse(pos, node, expected)) + return false; + + roles = std::static_pointer_cast(node); + return true; + }); + } +} + + +bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + bool alter; + if (ParserKeyword{"CREATE QUOTA"}.ignore(pos, expected)) + alter = false; + else if (ParserKeyword{"ALTER QUOTA"}.ignore(pos, expected)) + alter = true; + else + return false; + + bool if_exists = false; + bool if_not_exists = false; + bool or_replace = false; + if (alter) + { + if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected)) + if_exists = true; + } + else + { + if (ParserKeyword{"IF NOT EXISTS"}.ignore(pos, expected)) + if_not_exists = true; + else if (ParserKeyword{"OR REPLACE"}.ignore(pos, expected)) + or_replace = true; + } + + String name; + if (!parseIdentifierOrStringLiteral(pos, expected, name)) + return false; + + String new_name; + std::optional key_type; + std::vector all_limits; + std::shared_ptr roles; + + while (parseRenameTo(pos, expected, new_name, alter) || parseKeyType(pos, expected, key_type) + || parseAllLimits(pos, expected, all_limits, alter) || parseRoles(pos, expected, roles)) + ; + + auto query = std::make_shared(); + node = query; + + query->alter = alter; + query->if_exists = if_exists; + query->if_not_exists = if_not_exists; + query->or_replace = or_replace; + query->name = std::move(name); + query->new_name = std::move(new_name); + query->key_type = key_type; + query->all_limits = std::move(all_limits); + query->roles = std::move(roles); + + return true; +} +} diff --git a/dbms/src/Parsers/ParserCreateQuotaQuery.h b/dbms/src/Parsers/ParserCreateQuotaQuery.h new file mode 100644 index 00000000000..aef33f72e67 --- /dev/null +++ b/dbms/src/Parsers/ParserCreateQuotaQuery.h @@ -0,0 +1,31 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * CREATE QUOTA [IF NOT EXISTS | OR REPLACE] name + * [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] + * [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + * {[SET] MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = {number | ANY} } [,...] | + * [SET] TRACKING} [,...]] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + * + * ALTER QUOTA [IF EXISTS] name + * [RENAME TO new_name] + * [KEYED BY {'none' | 'user name' | 'ip address' | 'client key' | 'client key or user name' | 'client key or ip address'}] + * [FOR [RANDOMIZED] INTERVAL number {SECOND | MINUTE | HOUR | DAY} + * {[SET] MAX {{QUERIES | ERRORS | RESULT ROWS | RESULT BYTES | READ ROWS | READ BYTES | EXECUTION TIME} = {number | ANY} } [,...] | + * [SET] TRACKING | + * UNSET TRACKING} [,...]] + * [TO {role [,...] | ALL | ALL EXCEPT role [,...]}] + */ +class ParserCreateQuotaQuery : public IParserBase +{ +protected: + const char * getName() const override { return "CREATE QUOTA or ALTER QUOTA query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp new file mode 100644 index 00000000000..c6d5ff889fc --- /dev/null +++ b/dbms/src/Parsers/ParserDropAccessEntityQuery.cpp @@ -0,0 +1,45 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"DROP"}.ignore(pos, expected)) + return false; + + using Kind = ASTDropAccessEntityQuery::Kind; + Kind kind; + if (ParserKeyword{"QUOTA"}.ignore(pos, expected)) + kind = Kind::QUOTA; + else + return false; + + bool if_exists = false; + if (ParserKeyword{"IF EXISTS"}.ignore(pos, expected)) + if_exists = true; + + Strings names; + do + { + String name; + if (!parseIdentifierOrStringLiteral(pos, expected, name)) + return false; + + names.push_back(std::move(name)); + } + while (ParserToken{TokenType::Comma}.ignore(pos, expected)); + + auto query = std::make_shared(kind); + node = query; + + query->if_exists = if_exists; + query->names = std::move(names); + + return true; +} +} diff --git a/dbms/src/Parsers/ParserDropAccessEntityQuery.h b/dbms/src/Parsers/ParserDropAccessEntityQuery.h new file mode 100644 index 00000000000..f479e0d0add --- /dev/null +++ b/dbms/src/Parsers/ParserDropAccessEntityQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * DROP QUOTA [IF EXISTS] name [,...] + */ +class ParserDropAccessEntityQuery : public IParserBase +{ +protected: + const char * getName() const override { return "DROP QUOTA query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/dbms/src/Parsers/ParserQuery.cpp b/dbms/src/Parsers/ParserQuery.cpp index b7bdd517a43..a3bb652032e 100644 --- a/dbms/src/Parsers/ParserQuery.cpp +++ b/dbms/src/Parsers/ParserQuery.cpp @@ -9,6 +9,8 @@ #include #include #include +#include +#include namespace DB @@ -22,12 +24,16 @@ bool ParserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserUseQuery use_p; ParserSetQuery set_p; ParserSystemQuery system_p; + ParserCreateQuotaQuery create_quota_p; + ParserDropAccessEntityQuery drop_access_entity_p; bool res = query_with_output_p.parse(pos, node, expected) || insert_p.parse(pos, node, expected) || use_p.parse(pos, node, expected) || set_p.parse(pos, node, expected) - || system_p.parse(pos, node, expected); + || system_p.parse(pos, node, expected) + || create_quota_p.parse(pos, node, expected) + || drop_access_entity_p.parse(pos, node, expected); return res; } diff --git a/dbms/src/Parsers/ParserQueryWithOutput.cpp b/dbms/src/Parsers/ParserQueryWithOutput.cpp index 1c44c639848..d08ae984c90 100644 --- a/dbms/src/Parsers/ParserQueryWithOutput.cpp +++ b/dbms/src/Parsers/ParserQueryWithOutput.cpp @@ -14,6 +14,8 @@ #include #include #include +#include +#include namespace DB @@ -34,6 +36,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec ParserOptimizeQuery optimize_p; ParserKillQueryQuery kill_query_p; ParserWatchQuery watch_p; + ParserShowCreateAccessEntityQuery show_create_access_entity_p; + ParserShowQuotasQuery show_quotas_p; ASTPtr query; @@ -49,6 +53,7 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec analyze_syntax = true; bool parsed = select_p.parse(pos, query, expected) + || show_create_access_entity_p.parse(pos, query, expected) /// should be before `show_tables_p` || show_tables_p.parse(pos, query, expected) || table_p.parse(pos, query, expected) || describe_table_p.parse(pos, query, expected) @@ -60,7 +65,8 @@ bool ParserQueryWithOutput::parseImpl(Pos & pos, ASTPtr & node, Expected & expec || check_p.parse(pos, query, expected) || kill_query_p.parse(pos, query, expected) || optimize_p.parse(pos, query, expected) - || watch_p.parse(pos, query, expected); + || watch_p.parse(pos, query, expected) + || show_quotas_p.parse(pos, query, expected); if (!parsed) return false; diff --git a/dbms/src/Parsers/ParserRoleList.cpp b/dbms/src/Parsers/ParserRoleList.cpp new file mode 100644 index 00000000000..ac8914de776 --- /dev/null +++ b/dbms/src/Parsers/ParserRoleList.cpp @@ -0,0 +1,78 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +bool ParserRoleList::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + Strings roles; + bool current_user = false; + bool all_roles = false; + Strings except_roles; + bool except_current_user = false; + + bool except_mode = false; + while (true) + { + if (ParserKeyword{"NONE"}.ignore(pos, expected)) + { + } + else if (ParserKeyword{"CURRENT_USER"}.ignore(pos, expected) || + ParserKeyword{"currentUser"}.ignore(pos, expected)) + { + if (ParserToken{TokenType::OpeningRoundBracket}.ignore(pos, expected)) + { + if (!ParserToken{TokenType::ClosingRoundBracket}.ignore(pos, expected)) + return false; + } + if (except_mode && !current_user) + except_current_user = true; + else + current_user = true; + } + else if (ParserKeyword{"ALL"}.ignore(pos, expected)) + { + all_roles = true; + if (ParserKeyword{"EXCEPT"}.ignore(pos, expected)) + { + except_mode = true; + continue; + } + } + else + { + String name; + if (!parseIdentifierOrStringLiteral(pos, expected, name)) + return false; + if (except_mode && (boost::range::find(roles, name) == roles.end())) + except_roles.push_back(name); + else + roles.push_back(name); + } + + if (!ParserToken{TokenType::Comma}.ignore(pos, expected)) + break; + } + + if (all_roles) + { + current_user = false; + roles.clear(); + } + + auto result = std::make_shared(); + result->roles = std::move(roles); + result->current_user = current_user; + result->all_roles = all_roles; + result->except_roles = std::move(except_roles); + result->except_current_user = except_current_user; + node = result; + return true; +} + +} diff --git a/dbms/src/Parsers/ParserRoleList.h b/dbms/src/Parsers/ParserRoleList.h new file mode 100644 index 00000000000..eca205a748c --- /dev/null +++ b/dbms/src/Parsers/ParserRoleList.h @@ -0,0 +1,18 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses a string like this: + * {role|CURRENT_USER} [,...] | NONE | ALL | ALL EXCEPT {role|CURRENT_USER} [,...] + */ +class ParserRoleList : public IParserBase +{ +protected: + const char * getName() const { return "RoleList"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); +}; + +} diff --git a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp new file mode 100644 index 00000000000..661330ffd0b --- /dev/null +++ b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.cpp @@ -0,0 +1,47 @@ +#include +#include +#include +#include + + +namespace DB +{ +bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + if (!ParserKeyword{"SHOW CREATE"}.ignore(pos, expected)) + return false; + + using Kind = ASTShowCreateAccessEntityQuery::Kind; + Kind kind; + if (ParserKeyword{"QUOTA"}.ignore(pos, expected)) + kind = Kind::QUOTA; + else + return false; + + String name; + bool current_quota = false; + + if ((kind == Kind::QUOTA) && ParserKeyword{"CURRENT"}.ignore(pos, expected)) + { + /// SHOW CREATE QUOTA CURRENT + current_quota = true; + } + else if (parseIdentifierOrStringLiteral(pos, expected, name)) + { + /// SHOW CREATE QUOTA name + } + else + { + /// SHOW CREATE QUOTA + current_quota = true; + } + + auto query = std::make_shared(kind); + node = query; + + query->name = std::move(name); + query->current_quota = current_quota; + + return true; +} +} diff --git a/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.h b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.h new file mode 100644 index 00000000000..4572b54de27 --- /dev/null +++ b/dbms/src/Parsers/ParserShowCreateAccessEntityQuery.h @@ -0,0 +1,17 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * SHOW CREATE QUOTA [name | CURRENT] + */ +class ParserShowCreateAccessEntityQuery : public IParserBase +{ +protected: + const char * getName() const override { return "SHOW CREATE QUOTA query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/dbms/src/Parsers/ParserShowQuotasQuery.cpp b/dbms/src/Parsers/ParserShowQuotasQuery.cpp new file mode 100644 index 00000000000..69cbd352969 --- /dev/null +++ b/dbms/src/Parsers/ParserShowQuotasQuery.cpp @@ -0,0 +1,42 @@ +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +bool ParserShowQuotasQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) +{ + bool usage; + bool current; + if (ParserKeyword{"SHOW QUOTAS"}.ignore(pos, expected)) + { + usage = false; + current = false; + } + else if (ParserKeyword{"SHOW QUOTA USAGE"}.ignore(pos, expected)) + { + usage = true; + if (ParserKeyword{"ALL"}.ignore(pos, expected)) + { + current = false; + } + else + { + ParserKeyword{"CURRENT"}.ignore(pos, expected); + current = true; + } + } + else + return false; + + auto query = std::make_shared(); + query->usage = usage; + query->current = current; + node = query; + return true; +} +} diff --git a/dbms/src/Parsers/ParserShowQuotasQuery.h b/dbms/src/Parsers/ParserShowQuotasQuery.h new file mode 100644 index 00000000000..5b00b525f98 --- /dev/null +++ b/dbms/src/Parsers/ParserShowQuotasQuery.h @@ -0,0 +1,18 @@ +#pragma once + +#include + + +namespace DB +{ +/** Parses queries like + * SHOW QUOTAS + * SHOW QUOTA USAGE [CURRENT | ALL] + */ +class ParserShowQuotasQuery : public IParserBase +{ +protected: + const char * getName() const override { return "SHOW QUOTA query"; } + bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; +}; +} diff --git a/dbms/tests/integration/test_quota/configs/users.xml b/dbms/tests/integration/test_quota/configs/users.xml index 4412345a731..15a5364449b 100644 --- a/dbms/tests/integration/test_quota/configs/users.xml +++ b/dbms/tests/integration/test_quota/configs/users.xml @@ -12,6 +12,7 @@ default myQuota + true diff --git a/dbms/tests/integration/test_quota/test.py b/dbms/tests/integration/test_quota/test.py index 5dae1be5e57..e7caaf5cd06 100644 --- a/dbms/tests/integration/test_quota/test.py +++ b/dbms/tests/integration/test_quota/test.py @@ -48,6 +48,7 @@ def reset_quotas_and_usage_info(): try: yield finally: + instance.query("DROP QUOTA IF EXISTS qA, qB") copy_quota_xml('simpliest.xml') # To reset usage info. copy_quota_xml('normal_limits.xml') @@ -175,3 +176,76 @@ def test_reload_users_xml_by_timer(): # because config files are reload by timer only when the modification time is changed. copy_quota_xml('tiny_limits.xml', reload_immediately=False) assert_eq_with_retry(instance, query_from_system_quotas, "myQuota\te651da9c-a748-8703-061a-7e5e5096dae7\tusers.xml\tuser name\t['default']\t0\t[]\t[31556952]\t[0]\t[1]\t[1]\t[1]\t[0]\t[1]\t[0]\t[0]") + + +def test_dcl_introspection(): + assert instance.query("SHOW QUOTAS") == "myQuota\n" + assert instance.query("SHOW CREATE QUOTA myQuota") == "CREATE QUOTA myQuota KEYED BY \\'user name\\' FOR INTERVAL 1 YEAR MAX QUERIES = 1000, MAX READ ROWS = 1000 TO default\n" + expected_usage = "myQuota key=\\\\'default\\\\' interval=\[.*\] queries=0/1000 errors=0 result_rows=0 result_bytes=0 read_rows=0/1000 read_bytes=0 execution_time=0" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE CURRENT")) + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE ALL")) + + instance.query("SELECT * from test_table") + expected_usage = "myQuota key=\\\\'default\\\\' interval=\[.*\] queries=1/1000 errors=0 result_rows=50 result_bytes=200 read_rows=50/1000 read_bytes=200 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + # Add interval. + copy_quota_xml('two_intervals.xml') + assert instance.query("SHOW QUOTAS") == "myQuota\n" + assert instance.query("SHOW CREATE QUOTA myQuota") == "CREATE QUOTA myQuota KEYED BY \\'user name\\' FOR INTERVAL 1 YEAR MAX QUERIES = 1000, MAX READ ROWS = 1000, FOR RANDOMIZED INTERVAL 2 YEAR MAX RESULT BYTES = 30000, MAX READ BYTES = 20000, MAX EXECUTION TIME = 120 TO default\n" + expected_usage = "myQuota key=\\\\'default\\\\' interval=\[.*\] queries=1/1000 errors=0 result_rows=50 result_bytes=200 read_rows=50/1000 read_bytes=200 execution_time=.*\n"\ + "myQuota key=\\\\'default\\\\' interval=\[.*\] queries=0 errors=0 result_rows=0 result_bytes=0/30000 read_rows=0 read_bytes=0/20000 execution_time=0/120" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + # Drop interval, add quota. + copy_quota_xml('two_quotas.xml') + assert instance.query("SHOW QUOTAS") == "myQuota\nmyQuota2\n" + assert instance.query("SHOW CREATE QUOTA myQuota") == "CREATE QUOTA myQuota KEYED BY \\'user name\\' FOR INTERVAL 1 YEAR MAX QUERIES = 1000, MAX READ ROWS = 1000 TO default\n" + assert instance.query("SHOW CREATE QUOTA myQuota2") == "CREATE QUOTA myQuota2 KEYED BY \\'client key or user name\\' FOR RANDOMIZED INTERVAL 1 HOUR MAX RESULT ROWS = 4000, MAX RESULT BYTES = 400000, MAX READ ROWS = 4000, MAX READ BYTES = 400000, MAX EXECUTION TIME = 60, FOR INTERVAL 1 MONTH MAX EXECUTION TIME = 1800\n" + expected_usage = "myQuota key=\\\\'default\\\\' interval=\[.*\] queries=1/1000 errors=0 result_rows=50 result_bytes=200 read_rows=50/1000 read_bytes=200 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + +def test_dcl_management(): + copy_quota_xml('no_quotas.xml') + assert instance.query("SHOW QUOTAS") == "" + assert instance.query("SHOW QUOTA USAGE") == "" + + instance.query("CREATE QUOTA qA FOR INTERVAL 15 MONTH SET MAX QUERIES = 123 TO CURRENT_USER") + assert instance.query("SHOW QUOTAS") == "qA\n" + assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA KEYED BY \\'none\\' FOR INTERVAL 5 QUARTER MAX QUERIES = 123 TO default\n" + expected_usage = "qA key=\\\\'\\\\' interval=\[.*\] queries=0/123 errors=0 result_rows=0 result_bytes=0 read_rows=0 read_bytes=0 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + instance.query("SELECT * from test_table") + expected_usage = "qA key=\\\\'\\\\' interval=\[.*\] queries=1/123 errors=0 result_rows=50 result_bytes=200 read_rows=50 read_bytes=200 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + instance.query("ALTER QUOTA qA FOR INTERVAL 15 MONTH MAX QUERIES = 321, MAX ERRORS = 10, FOR INTERVAL 0.5 HOUR MAX EXECUTION TIME = 0.5") + assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA KEYED BY \\'none\\' FOR INTERVAL 30 MINUTE MAX EXECUTION TIME = 0.5, FOR INTERVAL 5 QUARTER MAX QUERIES = 321, MAX ERRORS = 10 TO default\n" + expected_usage = "qA key=\\\\'\\\\' interval=\[.*\] queries=0 errors=0 result_rows=0 result_bytes=0 read_rows=0 read_bytes=0 execution_time=.*/0.5\n"\ + "qA key=\\\\'\\\\' interval=\[.*\] queries=1/321 errors=0/10 result_rows=50 result_bytes=200 read_rows=50 read_bytes=200 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + instance.query("ALTER QUOTA qA FOR INTERVAL 15 MONTH UNSET TRACKING, FOR RANDOMIZED INTERVAL 16 MONTH SET TRACKING, FOR INTERVAL 1800 SECOND UNSET TRACKING") + assert instance.query("SHOW CREATE QUOTA qA") == "CREATE QUOTA qA KEYED BY \\'none\\' FOR RANDOMIZED INTERVAL 16 MONTH TRACKING TO default\n" + expected_usage = "qA key=\\\\'\\\\' interval=\[.*\] queries=0 errors=0 result_rows=0 result_bytes=0 read_rows=0 read_bytes=0 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + instance.query("SELECT * from test_table") + expected_usage = "qA key=\\\\'\\\\' interval=\[.*\] queries=1 errors=0 result_rows=50 result_bytes=200 read_rows=50 read_bytes=200 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + instance.query("ALTER QUOTA qA RENAME TO qB") + assert instance.query("SHOW CREATE QUOTA qB") == "CREATE QUOTA qB KEYED BY \\'none\\' FOR RANDOMIZED INTERVAL 16 MONTH TRACKING TO default\n" + expected_usage = "qB key=\\\\'\\\\' interval=\[.*\] queries=1 errors=0 result_rows=50 result_bytes=200 read_rows=50 read_bytes=200 execution_time=.*" + assert re.match(expected_usage, instance.query("SHOW QUOTA USAGE")) + + instance.query("DROP QUOTA qB") + assert instance.query("SHOW QUOTAS") == "" + assert instance.query("SHOW QUOTA USAGE") == "" + + +def test_users_xml_is_readonly(): + assert re.search("storage is readonly", instance.query_and_get_error("DROP QUOTA myQuota")) diff --git a/dbms/tests/queries/0_stateless/01033_quota_dcl.reference b/dbms/tests/queries/0_stateless/01033_quota_dcl.reference new file mode 100644 index 00000000000..7f92f992dd5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_quota_dcl.reference @@ -0,0 +1,2 @@ +default +CREATE QUOTA default KEYED BY \'user name\' FOR INTERVAL 1 HOUR TRACKING TO default, readonly diff --git a/dbms/tests/queries/0_stateless/01033_quota_dcl.sql b/dbms/tests/queries/0_stateless/01033_quota_dcl.sql new file mode 100644 index 00000000000..a1c7f1fc204 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_quota_dcl.sql @@ -0,0 +1,3 @@ +SHOW QUOTAS; +SHOW CREATE QUOTA default; +CREATE QUOTA q1; -- { serverError 497 } From 7783475aaafaadaa0dc417361f0cbf20d0214159 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 5 Dec 2019 21:53:26 +0300 Subject: [PATCH 75/77] delete useless test --- .../01030_incorrect_count_merge_tree.sql | 16 ---------------- 1 file changed, 16 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql deleted file mode 100644 index 7c496ab5ca6..00000000000 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.sql +++ /dev/null @@ -1,16 +0,0 @@ - -drop table if exists tst2; -create table tst2 -( - timestamp DateTime, - val Nullable(Int8) -) engine MergeTree partition by toYYYYMM(timestamp) ORDER by (timestamp); - -insert into tst2 values ('2018-02-01 00:00:00', 1), ('2018-02-02 00:00:00', 2); - -select count() from tst2; -select count() from tst2 where val is not null; -select count() from tst2 where timestamp is not null; -select count() from tst2 where timestamp > '2017-01-01 00:00:00'; - -drop table if exists tst2; From 826a6759745e74de50341bbe8531e2758f78da91 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 5 Dec 2019 21:53:44 +0300 Subject: [PATCH 76/77] delete useless reference --- .../0_stateless/01030_incorrect_count_merge_tree.reference | 4 ---- 1 file changed, 4 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference deleted file mode 100644 index 487b1165348..00000000000 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_merge_tree.reference +++ /dev/null @@ -1,4 +0,0 @@ -2 -2 -2 -2 From 5e60e0661f82062360aa0069acd9f06d1f3e767f Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 5 Dec 2019 21:54:18 +0300 Subject: [PATCH 77/77] delete useless reference --- ...t_count_summing_merge_tree_with_nullables.reference | 10 ---------- 1 file changed, 10 deletions(-) delete mode 100644 dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference diff --git a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference b/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference deleted file mode 100644 index 75378377541..00000000000 --- a/dbms/tests/queries/0_stateless/01030_incorrect_count_summing_merge_tree_with_nullables.reference +++ /dev/null @@ -1,10 +0,0 @@ -2 -2 -2 -2 -2 -2 -2 -2 -2 -2