diff --git a/src/Interpreters/Set.cpp b/src/Interpreters/Set.cpp index a4fea5dd705..bbb7fa4303a 100644 --- a/src/Interpreters/Set.cpp +++ b/src/Interpreters/Set.cpp @@ -596,6 +596,14 @@ BoolMask MergeTreeSetIndex::checkInRange(const std::vector & key_ranges, }; } +bool MergeTreeSetIndex::hasMonotonicFunctionsChain() const +{ + for (const auto & mapping : indexes_mapping) + if (!mapping.functions.empty()) + return true; + return false; +} + void ValueWithInfinity::update(const Field & x) { /// Keep at most one element in column. @@ -607,8 +615,11 @@ void ValueWithInfinity::update(const Field & x) const IColumn & ValueWithInfinity::getColumnIfFinite() const { +#ifndef NDEBUG if (type != NORMAL) throw Exception("Trying to get column of infinite type", ErrorCodes::LOGICAL_ERROR); +#endif + return *column; } diff --git a/src/Interpreters/Set.h b/src/Interpreters/Set.h index 90ff6c48dec..0f84c2f5da8 100644 --- a/src/Interpreters/Set.h +++ b/src/Interpreters/Set.h @@ -231,6 +231,8 @@ public: size_t size() const { return ordered_set.at(0)->size(); } + bool hasMonotonicFunctionsChain() const; + BoolMask checkInRange(const std::vector & key_ranges, const DataTypes & data_types); private: diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 3e0caa67518..5d799d257bc 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -98,8 +98,8 @@ void IMergeTreeDataPart::MinMaxIndex::update(const Block & block, const Names & for (size_t i = 0; i < column_names.size(); ++i) { - Field min_value; - Field max_value; + FieldRef min_value; + FieldRef max_value; const ColumnWithTypeAndName & column = block.getByName(column_names[i]); column.column->getExtremes(min_value, max_value); diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e994d254958..e755c4942a1 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -338,44 +338,6 @@ inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVi inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); } -FieldWithInfinity::FieldWithInfinity(const Field & field_) - : field(field_), - type(Type::NORMAL) -{ -} - -FieldWithInfinity::FieldWithInfinity(Field && field_) - : field(std::move(field_)), - type(Type::NORMAL) -{ -} - -FieldWithInfinity::FieldWithInfinity(const Type type_) - : type(type_) -{ -} - -FieldWithInfinity FieldWithInfinity::getMinusInfinity() -{ - return FieldWithInfinity(Type::MINUS_INFINITY); -} - -FieldWithInfinity FieldWithInfinity::getPlusInfinity() -{ - return FieldWithInfinity(Type::PLUS_INFINITY); -} - -bool FieldWithInfinity::operator<(const FieldWithInfinity & other) const -{ - return type < other.type || (type == other.type && type == Type::NORMAL && field < other.field); -} - -bool FieldWithInfinity::operator==(const FieldWithInfinity & other) const -{ - return type == other.type && (type != Type::NORMAL || field == other.field); -} - - /** Calculate expressions, that depend only on constants. * For index to work when something like "WHERE Date = toDate(now())" is written. */ @@ -480,24 +442,41 @@ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants } -static void applyFunction( +static Field applyFunctionForField( const FunctionBasePtr & func, - const DataTypePtr & arg_type, const Field & arg_value, - DataTypePtr & res_type, Field & res_value) + const DataTypePtr & arg_type, + const Field & arg_value) { - res_type = func->getReturnType(); - Block block { { arg_type->createColumnConst(1, arg_value), arg_type, "x" }, - { nullptr, res_type, "y" } + { nullptr, func->getReturnType(), "y" } }; func->execute(block, {0}, 1, 1); - - block.safeGetByPosition(1).column->get(0, res_value); + return (*block.safeGetByPosition(1).column)[0]; } +static FieldRef applyFunction(FunctionBasePtr & func, const DataTypePtr & current_type, const FieldRef & field) +{ + /// Fallback for fields without block reference. + if (field.isExplicit()) + return applyFunctionForField(func, current_type, field); + + String result_name = "_" + func->getName() + "_" + toString(field.column_idx); + size_t result_idx; + const auto & block = field.block; + if (!block->has(result_name)) + { + result_idx = block->columns(); + field.block->insert({nullptr, func->getReturnType(), result_name}); + func->execute(*block, {field.column_idx}, result_idx, block->rows()); + } + else + result_idx = block->getPositionByName(result_name); + + return {field.block, field.row_idx, result_idx}; +} void KeyCondition::traverseAST(const ASTPtr & node, const Context & context, Block & block_with_constants) { @@ -569,12 +548,8 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( return false; // Apply the next transformation step - DataTypePtr new_type; - applyFunction(a.function_base, out_type, out_value, new_type, out_value); - if (!new_type) - return false; - - out_type.swap(new_type); + out_value = applyFunctionForField(a.function_base, out_type, out_value); + out_type = a.function_base->getReturnType(); expr_name = a.result_name; // Transformation results in a key expression, accept @@ -957,8 +932,8 @@ String KeyCondition::toString() const template static BoolMask forAnyHyperrectangle( size_t key_size, - const Field * key_left, - const Field * key_right, + const FieldRef * key_left, + const FieldRef * key_right, bool left_bounded, bool right_bounded, std::vector & hyperrectangle, @@ -1049,8 +1024,8 @@ static BoolMask forAnyHyperrectangle( BoolMask KeyCondition::checkInRange( size_t used_key_size, - const Field * left_key, - const Field * right_key, + const FieldRef * left_key, + const FieldRef * right_key, const DataTypes & data_types, bool right_bounded, BoolMask initial_mask) const @@ -1102,19 +1077,12 @@ std::optional KeyCondition::applyMonotonicFunctionsChainToRange( return {}; } - /// Apply the function. - DataTypePtr new_type; if (!key_range.left.isNull()) - applyFunction(func, current_type, key_range.left, new_type, key_range.left); + key_range.left = applyFunction(func, current_type, key_range.left); if (!key_range.right.isNull()) - applyFunction(func, current_type, key_range.right, new_type, key_range.right); + key_range.right = applyFunction(func, current_type, key_range.right); - if (!new_type) - { - return {}; - } - - current_type.swap(new_type); + current_type = func->getReturnType(); if (!monotonicity.is_positive) key_range.swapLeftAndRight(); @@ -1220,8 +1188,8 @@ BoolMask KeyCondition::checkInHyperrectangle( BoolMask KeyCondition::checkInRange( size_t used_key_size, - const Field * left_key, - const Field * right_key, + const FieldRef * left_key, + const FieldRef * right_key, const DataTypes & data_types, BoolMask initial_mask) const { @@ -1231,8 +1199,8 @@ BoolMask KeyCondition::checkInRange( bool KeyCondition::mayBeTrueInRange( size_t used_key_size, - const Field * left_key, - const Field * right_key, + const FieldRef * left_key, + const FieldRef * right_key, const DataTypes & data_types) const { return checkInRange(used_key_size, left_key, right_key, data_types, true, BoolMask::consider_only_can_be_true).can_be_true; @@ -1241,7 +1209,7 @@ bool KeyCondition::mayBeTrueInRange( BoolMask KeyCondition::checkAfter( size_t used_key_size, - const Field * left_key, + const FieldRef * left_key, const DataTypes & data_types, BoolMask initial_mask) const { @@ -1251,7 +1219,7 @@ BoolMask KeyCondition::checkAfter( bool KeyCondition::mayBeTrueAfter( size_t used_key_size, - const Field * left_key, + const FieldRef * left_key, const DataTypes & data_types) const { return checkInRange(used_key_size, left_key, nullptr, data_types, false, BoolMask::consider_only_can_be_true).can_be_true; @@ -1382,4 +1350,13 @@ size_t KeyCondition::getMaxKeyColumn() const return res; } +bool KeyCondition::hasMonotonicFunctionsChain() const +{ + for (const auto & element : rpn) + if (!element.monotonic_functions_chain.empty() + || (element.set_index && element.set_index->hasMonotonicFunctionsChain())) + return true; + return false; +} + } diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 8667e0aea27..3a3768f0e4c 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -15,10 +15,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_TYPE_OF_FIELD; -} class IFunction; using FunctionBasePtr = std::shared_ptr; @@ -26,6 +22,33 @@ using FunctionBasePtr = std::shared_ptr; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +/** A field, that can be stored in two reperesenations: + * - A standalone field. + * - A field with reference to its position in a block. + * It's needed for execution of functions on ranges during + * index analysis. If function was executed once for field, + * its result would be cached for whole block for which field's reference points to. + */ +struct FieldRef : public Field +{ + FieldRef() = default; + + /// Create as explicit field without block. + template + FieldRef(T && value) : Field(std::forward(value)) {} + + /// Create as reference to field in block. + FieldRef(Block * block_, size_t row_idx_, size_t column_idx_) + : Field((*block_->getByPosition(column_idx_).column)[row_idx_]), + block(block_), row_idx(row_idx_), column_idx(column_idx_) {} + + bool isExplicit() const { return block == nullptr; } + + Block * block = nullptr; + size_t row_idx = 0; + size_t column_idx = 0; +}; + /** Range with open or closed ends; possibly unbounded. */ struct Range @@ -35,8 +58,8 @@ private: static bool less(const Field & lhs, const Field & rhs); public: - Field left; /// the left border, if any - Field right; /// the right border, if any + FieldRef left; /// the left border, if any + FieldRef right; /// the right border, if any bool left_bounded = false; /// bounded at the left bool right_bounded = false; /// bounded at the right bool left_included = false; /// includes the left border, if any @@ -46,11 +69,11 @@ public: Range() {} /// One point. - Range(const Field & point) + Range(const FieldRef & point) : left(point), right(point), left_bounded(true), right_bounded(true), left_included(true), right_included(true) {} /// A bounded two-sided range. - Range(const Field & left_, bool left_included_, const Field & right_, bool right_included_) + Range(const FieldRef & left_, bool left_included_, const FieldRef & right_, bool right_included_) : left(left_), right(right_), left_bounded(true), right_bounded(true), left_included(left_included_), right_included(right_included_) @@ -58,7 +81,7 @@ public: shrinkToIncludedIfPossible(); } - static Range createRightBounded(const Field & right_point, bool right_included) + static Range createRightBounded(const FieldRef & right_point, bool right_included) { Range r; r.right = right_point; @@ -68,7 +91,7 @@ public: return r; } - static Range createLeftBounded(const Field & left_point, bool left_included) + static Range createLeftBounded(const FieldRef & left_point, bool left_included) { Range r; r.left = left_point; @@ -84,7 +107,7 @@ public: */ void shrinkToIncludedIfPossible() { - if (left_bounded && !left_included) + if (left.isExplicit() && left_bounded && !left_included) { if (left.getType() == Field::Types::UInt64 && left.get() != std::numeric_limits::max()) { @@ -97,7 +120,7 @@ public: left_included = true; } } - if (right_bounded && !right_included) + if (right.isExplicit() && right_bounded && !right_included) { if (right.getType() == Field::Types::UInt64 && right.get() != std::numeric_limits::min()) { @@ -120,13 +143,13 @@ public: } /// x contained in the range - bool contains(const Field & x) const + bool contains(const FieldRef & x) const { return !leftThan(x) && !rightThan(x); } /// x is to the left - bool rightThan(const Field & x) const + bool rightThan(const FieldRef & x) const { return (left_bounded ? !(less(left, x) || (left_included && equals(x, left))) @@ -134,7 +157,7 @@ public: } /// x is to the right - bool leftThan(const Field & x) const + bool leftThan(const FieldRef & x) const { return (right_bounded ? !(less(x, right) || (right_included && equals(x, right))) @@ -195,42 +218,6 @@ public: String toString() const; }; - -/// Class that extends arbitrary objects with infinities, like +-inf for floats -class FieldWithInfinity -{ -public: - enum Type - { - MINUS_INFINITY = -1, - NORMAL = 0, - PLUS_INFINITY = 1 - }; - - explicit FieldWithInfinity(const Field & field_); - FieldWithInfinity(Field && field_); - - static FieldWithInfinity getMinusInfinity(); - static FieldWithInfinity getPlusInfinity(); - - bool operator<(const FieldWithInfinity & other) const; - bool operator==(const FieldWithInfinity & other) const; - - Field getFieldIfFinite() const - { - if (type != NORMAL) - throw Exception("Trying to get field of infinite type", ErrorCodes::BAD_TYPE_OF_FIELD); - return field; - } - -private: - Field field; - Type type; - - FieldWithInfinity(const Type type_); -}; - - /** Condition on the index. * * Consists of the conditions for the key belonging to all possible ranges or sets, @@ -261,8 +248,8 @@ public: /// one of the resulting mask components (see BoolMask::consider_only_can_be_XXX). BoolMask checkInRange( size_t used_key_size, - const Field * left_key, - const Field * right_key, + const FieldRef * left_key, + const FieldRef* right_key, const DataTypes & data_types, BoolMask initial_mask = BoolMask(false, false)) const; @@ -270,7 +257,7 @@ public: /// left_key must contain all the fields in the sort_descr in the appropriate order. BoolMask checkAfter( size_t used_key_size, - const Field * left_key, + const FieldRef * left_key, const DataTypes & data_types, BoolMask initial_mask = BoolMask(false, false)) const; @@ -278,15 +265,15 @@ public: /// This is more efficient than checkInRange(...).can_be_true. bool mayBeTrueInRange( size_t used_key_size, - const Field * left_key, - const Field * right_key, + const FieldRef * left_key, + const FieldRef * right_key, const DataTypes & data_types) const; /// Same as checkAfter, but calculate only may_be_true component of a result. /// This is more efficient than checkAfter(...).can_be_true. bool mayBeTrueAfter( size_t used_key_size, - const Field * left_key, + const FieldRef * left_key, const DataTypes & data_types) const; /// Checks that the index can not be used. @@ -295,6 +282,8 @@ public: /// Get the maximum number of the key element used in the condition. size_t getMaxKeyColumn() const; + bool hasMonotonicFunctionsChain() const; + /// Impose an additional condition: the value in the column `column` must be in the range `range`. /// Returns whether there is such a column in the key. bool addCondition(const String & column, const Range & range); @@ -374,8 +363,8 @@ public: private: BoolMask checkInRange( size_t used_key_size, - const Field * left_key, - const Field * right_key, + const FieldRef * left_key, + const FieldRef * right_key, const DataTypes & data_types, bool right_bounded, BoolMask initial_mask) const; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 370286117ae..816af8db3e9 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1201,11 +1201,33 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( * If fits, split it into smaller ones and put them on the stack. If not, discard it. * If the segment is already of one mark length, add it to response and discard it. */ - std::vector ranges_stack{ {0, marks_count} }; + std::vector ranges_stack = { {0, marks_count} }; + + std::function create_field_ref; + /// If there are no monotonic functions, there is no need to save block reference. + /// Passing explicit field to FieldRef allows to optimize ranges and shows better performance. + if (key_condition.hasMonotonicFunctionsChain()) + { + auto index_block = std::make_shared(); + for (size_t i = 0; i < used_key_size; ++i) + index_block->insert({index[i], data.primary_key_data_types[i], data.primary_key_columns[i]}); + + create_field_ref = [index_block](size_t row, size_t column, FieldRef & field) + { + field = {index_block.get(), row, column}; + }; + } + else + { + create_field_ref = [&index](size_t row, size_t column, FieldRef & field) + { + index[column]->get(row, field); + }; + } /// NOTE Creating temporary Field objects to pass to KeyCondition. - Row index_left(used_key_size); - Row index_right(used_key_size); + std::vector index_left(used_key_size); + std::vector index_right(used_key_size); while (!ranges_stack.empty()) { @@ -1216,7 +1238,7 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( if (range.end == marks_count && !has_final_mark) { for (size_t i = 0; i < used_key_size; ++i) - index[i]->get(range.begin, index_left[i]); + create_field_ref(range.begin, i, index_left[i]); may_be_true = key_condition.mayBeTrueAfter( used_key_size, index_left.data(), data.primary_key_data_types); @@ -1228,8 +1250,8 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( for (size_t i = 0; i < used_key_size; ++i) { - index[i]->get(range.begin, index_left[i]); - index[i]->get(range.end, index_right[i]); + create_field_ref(range.begin, i, index_left[i]); + create_field_ref(range.end, i, index_right[i]); } may_be_true = key_condition.mayBeTrueInRange( @@ -1254,9 +1276,9 @@ MarkRanges MergeTreeDataSelectExecutor::markRangesFromPKRange( size_t end; for (end = range.end; end > range.begin + step; end -= step) - ranges_stack.push_back(MarkRange(end - step, end)); + ranges_stack.emplace_back(end - step, end); - ranges_stack.push_back(MarkRange(range.begin, end)); + ranges_stack.emplace_back(range.begin, end); } } } diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index 122f038fee6..220fc70c549 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -100,8 +100,8 @@ void MergeTreeIndexAggregatorMinMax::update(const Block & block, size_t * pos, s size_t rows_read = std::min(limit, block.rows() - *pos); - Field field_min; - Field field_max; + FieldRef field_min; + FieldRef field_max; for (size_t i = 0; i < index.columns.size(); ++i) { const auto & column = block.getByName(index.columns[i]).column; diff --git a/tests/performance/set_index.xml b/tests/performance/set_index.xml index 090d8ac8c08..f158c481d93 100644 --- a/tests/performance/set_index.xml +++ b/tests/performance/set_index.xml @@ -14,7 +14,14 @@ - SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM zeros(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' + SELECT count() FROM test_in WHERE a IN (SELECT rand(1) FROM numbers(100000)) SETTINGS max_rows_to_read = 1, read_overflow_mode = 'break' + + SELECT count() FROM test_in WHERE toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(100000)) settings max_rows_to_read=1, read_overflow_mode='break' + + + SELECT count() FROM test_in WHERE -toInt64(a) IN (SELECT toInt64(rand(1)) FROM numbers(100000)) settings max_rows_to_read=1, read_overflow_mode='break' + + SELECT count() FROM test_in WHERE -toInt64(a) NOT IN (SELECT toInt64(rand(1)) FROM numbers(100000)) settings max_rows_to_read=1, read_overflow_mode='break' SELECT count() FROM numbers(1000) WHERE toString(number) IN ('41577', '83972', '51697', '50014', '37553', '93459', '87438', '95971', '83186', '74326', '67871', '50406', '83678', '29655', '18580', '83905', '61518', '29059', '56700', '82787', '98672', '30884', '81822', '39850', '80852', '57627', '91346', '64522', '17781', '49467', '41099', '41929', '85618', '91389', '68564', '91769', '81219', '52218', '37220', '97097', '2129', '9886', '52049', '34847', '25364', '36429', '76897', '71868', '58121', '71199', '84819', '69991', '34046', '64507', '34892', '24228', '36986', '28588', '51159', '53444', '80531', '9941', '20256', '48103', '32565', '62890', '5379', '60302', '46434', '3205', '18821', '31030', '19794', '71557', '71703', '15024', '14004', '82164', '95659', '40227', '83358', '24395', '9610', '19814', '48491', '66412', '16012', '71586', '42143', '51103', '24463', '89949', '35694', '39193', '63904', '40489', '77144', '94014', '84836', '9980', '46554', '43905', '25588', '25205', '72624', '10249', '35888', '98478', '99030', '26834', '31', '81499', '14847', '82997', '92357', '92893', '17426', '56630', '22252', '68119', '62710', '8740', '82144', '79916', '23391', '30192', '99271', '96435', '44237', '98327', '69481', '16691', '13643', '84554', '38571', '70926', '99283', '79000', '20926', '86495', '4834', '1222', '39486', '57697', '58002', '40790', '15623', '3999', '31515', '12694', '26143', '35951', '54085', '97534', '35329', '73535', '88715', '29572', '75799', '45166', '32066', '48023', '69523', '93150', '8740', '96790', '15534', '63252', '5142', '67045', '93992', '16663', '292', '63924', '6588', '12190', '31506', '69590', '35394', '55168', '65223', '79183', '32600', '69676', '28316', '72111', '53531', '15073', '41127', '73451', '24725', '61647', '65315', '41143', '26493', '95608', '34407', '76098', '53105', '83691', '48755', '35696', '62587', '81826', '3963', '45766', '82751', '12430', '97685', '29919', '78155', '71636', '50215', '89734', '9892', '47151', '54855', '3428', '9712', '52592', '2403', '79602', '81243', '79859', '57361', '82000', '42107', '28860', '99591', '28296', '57337', '64969', '32332', '25535', '30924', '21313', '32554', '17342', '87311', '19825', '24898', '61323', '83209', '79322', '79009', '50746', '33396', '62033', '16548', '17427', '24073', '34640', '52368', '4724', '80408', '40', '33787', '16666', '19665', '86751', '27264', '2241', '88134', '53566', '10589', '79711', '92823', '58972', '91767', '60885', '51659', '7867', '96849', '30360', '20914', '9584', '1250', '22871', '23282', '99312', '4683', '33429', '68361', '82614', '81440', '47863', '69790', '11968', '75210', '66854', '37002', '61142', '71514', '1588', '42336', '11069', '26291', '2261', '71056', '13492', '9133', '91216', '72207', '71586', '86535', '83898', '24392', '45384', '48545', '61972', '503', '80180', '35834', '97025', '70411', '55039', '35430', '27631', '82533', '96831', '74077', '42533', '14451', '26943', '53783', '69489', '71969', '8432', '37230', '61348', '19472', '59115', '9886', '50951', '57109', '7141', '1902', '84130', '4323', '55889', '47784', '2220', '75988', '66988', '63721', '8131', '95601', '95207', '2311', '26541', '50991', '6717', '2969', '71857', '51034', '65958', '94716', '90275', '21012', '46859', '7984', '31131', '46457', '69578', '44540', '7294', '80117', '9925', '60155', '90608', '82684', '32193', '87071', '28006', '87604', '24501', '79087', '2848', '29237', '11221', '81319', '40966', '87641', '35325', '78705', '88636', '78717', '62831', '56390', '99271', '43821', '14453', '17923', '62695', '77322', '21038', '67677', '41271', '4376', '65426', '46091', '19887', '97251', '55583', '58763', '3826', '35037', '73533', '64267', '82319', '9836', '42622', '96829', '16363', '10455', '49290', '99992', '98229', '66356', '59087', '73998', '25986', '4279', '56790', '69540', '588', '36620', '60358', '45056', '89297', '42740', '8323', '19245', '82417', '41431', '699', '11554', '73910', '44491', '56019', '68901', '45816', '68126', '89379', '23885', '13263', '56395', '73130', '19089', '23771', '10335', '48547', '16903', '6453', '33560', '89668', '38159', '43177', '90655', '49712', '62', '66920', '34180', '12150', '48564', '39538', '85026', '87195', '14928', '8956', '71157', '53287', '39161', '67583', '83309', '92054', '86977', '56188', '15229', '88170', '60894', '58497', '89254', '40082', '86890', '60161', '97291', '45878', '23368', '14577', '92870', '37017', '97356', '99426', '76061', '89186', '99751', '85153', '61580', '39360', '90107', '25603', '26798', '76224', '6469', '7912', '69838', '16404', '67497', '28965', '80836', '80365', '91249', '48713', '17113', '33090', '40793', '70450', '66689', '83698', '17802', '43869', '13355', '18959', '79411', '87930', '9265', '37504', '44876', '97234', '94149', '35040', '22049', '49248', '6535', '36080', '28346', '94437', '78319', '17961', '89056', '56161', '35810', '41632', '45494', '53351', '89729', '99510', '51584', '59688', '6193', '70809', '51093', '92589', '90247', '34910', '78235', '17362', '49423', '63324', '525', '37638', '72325', '89356', '15298', '59116', '17848', '65429', '27029', '84781', '70247', '8825', '35082', '70451', '22522', '58125', '91879', '90531', '2478', '463', '37902', '54405', '87267', '72688', '22803', '33134', '35177', '84551', '44974', '88375', '76407', '27774', '33849', '19915', '82014', '80434', '26380', '48777', '53811', '14838', '26829', '56441', '99869', '49574', '85476', '19723', '16907', '4018', '37338', '78510', '47912', '13030', '65277', '95716', '67363', '21393', '89887', '78842', '81650', '903', '17436', '30704', '49223', '27198', '25500', '52214', '54258', '70082', '53950', '49312', '43615', '99473', '94348', '53661', '96213', '96346', '62010', '38268', '32861', '75660', '10392', '89491', '68335', '29817', '88706', '24184', '36298', '43440', '21626', '26535', '44560', '46363', '12534', '99070', '95606', '33714', '73070', '8303', '29853', '23014', '99982', '4530', '14955', '45803', '50', '90750', '30394', '81276', '95563', '47314', '58520', '91299', '88944', '54402', '67405', '29253', '47079', '71734', '99728', '17652', '13307', '35556', '18962', '26780', '17771', '53712', '60055', '37628', '35830', '90739', '61151', '41309', '27652', '3051', '53167', '98417', '19382', '36833', '75085', '65374', '87732', '30352', '31776', '32765', '97565', '92199', '49050', '29503', '51024', '18834', '8515', '24069', '96216', '10777', '90680', '18974', '68884', '85305', '36007', '56707', '4212', '47352', '34426', '13185', '92939', '95782', '70577', '58080', '98279', '3906', '5065', '56896', '16382', '31273', '17117', '98602', '12786', '24086', '63970', '72756', '35798', '82367', '7356', '53398', '68503', '2962', '16425', '67334', '68461', '65439', '15620', '70906', '29649', '46461', '74602', '38012', '71714', '16825', '89480', '53386', '88532', '35104', '28556', '82120', '23155', '23347', '24797', '60061', '54962', '99427', '82248', '82447', '39968', '63727', '27431', '81511', '91168', '71425', '80740', '84127', '40717', '15503', '15419', '46594', '61263', '19212', '53175', '70724', '74445', '23034', '71818', '40246', '18886', '53066', '4880', '83701', '86107', '87862', '44751', '392', '73440', '90291', '93395', '20894', '38463', '32664', '55158', '20090', '50004', '79070', '98471', '85478', '96615', '68149', '78334', '97752', '73207', '71678', '91238', '96757', '82598', '194', '35797', '45120', '60782', '28721', '17676', '78066', '60957', '11826', '51563', '50516', '16485', '47053', '31738', '48923', '23554', '96850', '42033', '73701', '78607', '45979', '54571', '12415', '31693', '15356', '36902', '9126', '3767', '3295', '90402', '24005', '95350', '67033', '49137', '72606', '51899', '17522', '31957', '44641', '53982', '23767', '68257', '15766', '19995', '2107', '48788', '11765', '91055', '46576', '54651', '50381', '62827', '73636', '46606', '98753', '37631', '70441', '87916', '66983', '33870', '31125', '12904', '57040', '4874', '58632', '42037', '18782', '5998', '18974', '57949', '81010', '90407', '99874', '20462', '89949', '10952', '71454', '95130', '46115', '3518', '13384', '69039', '79482', '22076', '59782', '32042', '40930', '60243', '29298', '6790', '46985', '44398', '85631', '14380', '66179', '2629', '32126', '49833', '14118', '58492', '31493', '81172', '96638', '8745', '89663', '76842', '78633', '41373', '83721', '42886', '11123', '32739', '11051', '1303', '92314', '83324', '85600', '44276', '69064', '56125', '84650', '31028', '12628', '14502', '64764', '39405', '44855', '79046', '51716', '46824', '83389', '1941', '1257', '9280', '73176', '84729', '2579', '63366', '22606', '35541', '51096', '13447', '18355', '68037', '28436', '94116', '81070', '78355', '67897', '5296', '32742', '77645', '91853', '18767', '67949', '40963', '5792', '17278', '25597', '41884', '80829', '7099', '18645', '60295', '12082', '81800', '78415', '18082', '38789', '16295', '72377', '74949', '55583', '66853', '15402', '72977', '15123', '99434', '34999', '21687', '76049', '42987', '83748', '88256', '66688', '21766', '20304', '29271', '10069', '19822', '11792', '42526', '74143', '17289', '30253', '6367', '20888', '12975', '94073', '98639', '30134', '26320', '65507', '69002', '53120', '4550', '38893', '18954', '38283', '54863', '17698', '99670', '10521', '92467', '60994', '18052', '48673', '35811', '87282', '62706', '16061', '53112', '22652', '37780', '55662', '26331', '49410', '79074', '10623', '69577', '79613', '9491', '31229', '43922', '84231', '58409', '36386', '46875', '74431', '76735', '38776', '23350', '7314', '9079', '51519', '98544', '70216', '63380', '90381', '1295', '46901', '58225', '55339', '89918', '75522', '35431', '89460', '49552', '89302', '23068', '28493', '3042', '25194', '59520', '9810', '95706', '81297', '89638', '54794', '94527', '45262', '97932', '78685', '6947', '22818', '48700', '9153', '12289', '22011', '58825', '93854', '65438', '4509', '33741', '28208', '69061', '48578', '40247', '77725', '31837', '39003', '69363', '78113', '76398', '97262', '67795',